From 58011f03039cdc180d931cdbd238130d39252d9f Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Thu, 7 May 2020 14:59:11 -0700 Subject: [PATCH 1/9] use the full corr matrix' --- .../com/salesforce/op/ModelInsights.scala | 8 +-- .../DerivedFeatureFilterUtils.scala | 20 ++++++- .../impl/preparators/SanityChecker.scala | 29 ++++++--- .../preparators/SanityCheckerMetadata.scala | 59 +++++++++++-------- .../com/salesforce/op/ModelInsightsTest.scala | 2 +- .../SanityCheckerMetadataTest.scala | 11 ++-- .../impl/preparators/SanityCheckerTest.scala | 34 ++++++----- 7 files changed, 100 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/ModelInsights.scala b/core/src/main/scala/com/salesforce/op/ModelInsights.scala index 08645260b4..34273bc625 100644 --- a/core/src/main/scala/com/salesforce/op/ModelInsights.scala +++ b/core/src/main/scala/com/salesforce/op/ModelInsights.scala @@ -634,7 +634,7 @@ case object ModelInsights { derivedFeatureGroup = h.grouping, derivedFeatureValue = h.indicatorValue, excluded = Option(s.dropped.contains(h.columnName)), - corr = getCorr(s.correlationsWLabel, h.columnName), + corr = getCorr(s.correlations, h.columnName), cramersV = catGroupIndex.map(i => s.categoricalStats(i).cramersV), mutualInformation = catGroupIndex.map(i => s.categoricalStats(i).mutualInfo), pointwiseMutualInformation = (catGroupIndex, catIndexWithinGroup) match { @@ -743,11 +743,7 @@ case object ModelInsights { if (index >= 0) values.mapValues(_ (index)) else Map.empty private def getCorr(corr: Correlations, name: String): Option[Double] = { - getIfExists(corr.featuresIn.indexOf(name), corr.values).orElse { - val j = corr.nanCorrs.indexOf(name) - if (j >= 0) Option(Double.NaN) - else None - } + getIfExists(corr.featuresIn.indexOf(name), corr.valuesWithLabel) } private[op] def descaleLRContrib( diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala index 8474aa0385..c4e0234d93 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala @@ -36,6 +36,7 @@ import com.salesforce.op.utils.spark.RichMetadata._ import org.apache.log4j.Level import org.apache.spark.ml.linalg.{Vectors => NewVectors} import org.apache.spark.ml.param.{BooleanParam, DoubleParam, Param, Params} +import org.apache.spark.mllib.linalg.Matrix import org.apache.spark.mllib.stat.MultivariateStatisticalSummary import org.apache.spark.sql.types.Metadata @@ -98,7 +99,8 @@ object DerivedFeatureFilterUtils { labelNameAndIndex: Option[(String, Int)] = None, corrsWithLabel: Array[Double] = Array.empty, corrIndices: Array[Int] = Array.empty, - categoricalStats: Array[CategoricalGroupStats] = Array.empty + categoricalStats: Array[CategoricalGroupStats] = Array.empty, + corrMatrix: Option[Matrix] = None ): Array[ColumnStatistics] = { // precompute all statistics to avoid rebuilding the vectors every time val means = statsSummary.mean @@ -108,7 +110,7 @@ object DerivedFeatureFilterUtils { val variances = statsSummary.variance val cramersVMap = categoricalStats.flatMap(f => f.categoricalFeatures.map(c => c -> f.cramersV)) .toMap[String, Double] - val numCorrIndices = corrIndices.length + val featureCorrs = corrMatrix.map(_.rowIter.map(_.toArray).toArray) def maxByParent(seq: Seq[(String, Double)]) = seq.groupBy(_._1).map { case (k, v) => // Filter out the NaNs because max(3.4, NaN) = NaN, and we still want the keep the largest correlation @@ -174,6 +176,10 @@ object DerivedFeatureFilterUtils { case ind => Option(corrsWithLabel(ind)) }, cramersV = cramersVMap.get(name), + featureCorrs = corrIndices.indexOf(i) match { + case -1 => Seq.empty + case ind => featureCorrs.getOrElse(Array.empty).map(_.apply(ind)) + }, parentCorr = getParentValue(col, corrParent, corrParentNoKeys), parentCramersV = getParentValue(col, cramersVParent, cramersVParentNoKeys), maxRuleConfidences = maxRuleConfMap.getOrElse(name, Seq.empty), @@ -195,6 +201,7 @@ object DerivedFeatureFilterUtils { variance = variances(labelColumnIndex), corrLabel = None, cramersV = None, + featureCorrs = Seq.empty, parentCorr = None, parentCramersV = None, maxRuleConfidences = Seq.empty, @@ -214,6 +221,7 @@ object DerivedFeatureFilterUtils { * @param minVariance Min variance for dropping features * @param minCorrelation Min correlation with label for dropping features * @param maxCorrelation Max correlation with label for dropping features + * @param maxFeatureCorr Max correlation between features for dropping the later features * @param maxCramersV Max Cramer's V for dropping categorical features * @param maxRuleConfidence Max allowed confidence of association rules for dropping features * @param minRequiredRuleSupport Threshold for association rule @@ -229,6 +237,7 @@ object DerivedFeatureFilterUtils { minVariance: Double, minCorrelation: Double = 0.0, maxCorrelation: Double = 1.0, + maxFeatureCorr: Double = 1.0, maxCramersV: Double = 1.0, maxRuleConfidence: Double = 1.0, minRequiredRuleSupport: Double = 1.0, @@ -256,6 +265,7 @@ object DerivedFeatureFilterUtils { minVariance = minVariance, minCorrelation = minCorrelation, maxCorrelation = maxCorrelation, + maxFeatureCorr = maxFeatureCorr, maxCramersV = maxCramersV, maxRuleConfidence = maxRuleConfidence, minRequiredRuleSupport = minRequiredRuleSupport, @@ -314,6 +324,7 @@ private[op] case class ColumnStatistics cramersV: Option[Double], parentCorr: Option[Double], parentCramersV: Option[Double], + featureCorrs: Seq[Double], // Need to be able to hold up to two maxRuleConfidences or supports for the case of nullIndicator columns coming // from non-categorical features (since they will correspond to a 2x2 contingency matrix) maxRuleConfidences: Seq[Double], @@ -341,6 +352,7 @@ private[op] case class ColumnStatistics minVariance: Double, maxCorrelation: Double, minCorrelation: Double, + maxFeatureCorr: Double, maxCramersV: Double, maxRuleConfidence: Double, minRequiredRuleSupport: Double, @@ -361,6 +373,10 @@ private[op] case class ColumnStatistics corrLabel.filter(Math.abs(_) > maxCorrelation).map(corr => s"correlation $corr higher than max correlation $maxCorrelation" ), + column.flatMap{ case cl => featureCorrs.take(cl.index).find(Math.abs(_) > maxFeatureCorr).map(corr => + s"this feature has correlations $corr with other features higher than max feature-feature" + + s" correlation $maxFeatureCorr") + }, cramersV.filter(_ > maxCramersV).map(cv => s"Cramer's V $cv higher than max Cramer's V $maxCramersV" ), diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala index 9b30ff1cc8..ffbc48e220 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala @@ -99,6 +99,15 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { def setMaxCorrelation(value: Double): this.type = set(maxCorrelation, value) def getMaxCorrelation: Double = $(maxCorrelation) + final val maxFeatureCorr = new DoubleParam( + parent = this, name = "maxFeatureCorr", + doc = "Maximum correlation (absolute value) allowed between a feature two feature vectors which will" + + " both be kept. When this value is exceeded the second feature in the correlated pair will be dropped", + isValid = ParamValidators.inRange(lowerBound = 0.0, upperBound = 1.0, lowerInclusive = true, upperInclusive = true) + ) + def setMaxFeatureCorr(value: Double): this.type = set(maxCorrelation, value) + def getMaxFeatureCorr: Double = $(maxCorrelation) + final val minCorrelation = new DoubleParam( parent = this, name = "minCorrelation", doc = "Minimum correlation (absolute value) allowed between a feature in the feature vector and the label", @@ -190,6 +199,7 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { sampleUpperLimit -> SanityChecker.SampleUpperLimit, maxCorrelation -> SanityChecker.MaxCorrelation, minCorrelation -> SanityChecker.MinCorrelation, + maxFeatureCorr -> SanityChecker.MaxFeatureCorr, minVariance -> SanityChecker.MinVariance, maxCramersV -> SanityChecker.MaxCramersV, removeBadFeatures -> SanityChecker.RemoveBadFeatures, @@ -280,7 +290,7 @@ class SanityChecker(uid: String = UID[SanityChecker]) val colsCleaned = cols.map(_._2).filterNot(c => repeats.contains(c.index)) (group, colsCleaned.map(_.makeColName()), colsCleaned.map(_.index), colsCleaned.exists(_.hasParentOfSubType[MultiPickList])) - } + } colIndicesByGrouping.map { case (group, colNames, valueIndices, isMpl) => @@ -443,13 +453,13 @@ class SanityChecker(uid: String = UID[SanityChecker]) else ((0 until featureSize + 1).toArray, vectorRows) val numCorrIndices = corrIndices.length - // TODO: We are still calculating the full correlation matrix when featureLabelCorrOnly is false, but are not - // TODO: storing it anywhere. If we want access to the inter-feature correlations then need to refactor this a bit. - val corrsWithLabel = if ($(featureLabelCorrOnly)) { - OpStatistics.computeCorrelationsWithLabel(vectorRowsForCorr, colStats, count) + val (corrMatrix, corrsWithLabel) = if ($(featureLabelCorrOnly)) { + None -> OpStatistics.computeCorrelationsWithLabel(vectorRowsForCorr, colStats, count) + } + else { + val matrix = Statistics.corr(vectorRowsForCorr, getCorrelationType.sparkName) + Option(matrix) -> matrix.rowIter.map(_.apply(numCorrIndices - 1)).toArray } - else Statistics.corr(vectorRowsForCorr, getCorrelationType.sparkName).rowIter - .map(_.apply(numCorrIndices - 1)).toArray // Only calculate this if the label is categorical, so ignore if user has flagged label as not categorical val categoricalStats = @@ -467,7 +477,8 @@ class SanityChecker(uid: String = UID[SanityChecker]) Option((in1.name, featureSize)), // label column goes at end of vector corrsWithLabel, corrIndices, - categoricalStats + categoricalStats, + corrMatrix ) stats.foreach { stat => logInfo(stat.toString) } @@ -478,6 +489,7 @@ class SanityChecker(uid: String = UID[SanityChecker]) $(minVariance), $(minCorrelation), $(maxCorrelation), + $(maxFeatureCorr), $(maxCramersV), $(maxRuleConfidence), $(minRequiredRuleSupport), @@ -542,6 +554,7 @@ object SanityChecker { val SampleLowerLimit = 1E3.toInt val SampleUpperLimit = 1E6.toInt val MaxCorrelation = 0.95 + val MaxFeatureCorr = 0.999 val MinCorrelation = 0.0 val MinVariance = 1E-5 val MaxCramersV = 0.95 diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala index 1ec85660f4..e2b9753805 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala @@ -43,8 +43,7 @@ import scala.util.{Failure, Success, Try} * Contains all names for sanity checker metadata */ case object SanityCheckerNames extends DerivedFeatureFilterNames { - val CorrelationsWLabel: String = "correlationsWithLabel" - val CorrelationsWLabelIsNaN: String = "correlationsWithLabelIsNaN" + val Correlations: String = "correlations" val CorrelationType: String = "correlationType" val CategoricalStats: String = "categoricalStats" val CategoricalFeatures: String = "categoricalFeatures" @@ -58,7 +57,8 @@ case object SanityCheckerNames extends DerivedFeatureFilterNames { val Support: String = "support" val CountMatrix: String = "countMatrix" val FeaturesIn: String = "features" - val Values: String = "values" + val ValuesLabel: String = "valuesLabel" + val ValuesFeatures: String = "valuesFeatures" val NumNonZeros = "numNonZeros" val NumNull = "number of nulls" } @@ -66,7 +66,7 @@ case object SanityCheckerNames extends DerivedFeatureFilterNames { /** * Case class to convert to and from [[SanityChecker]] summary metadata * - * @param correlationsWLabel feature correlations with label + * @param correlations feature correlations with label * @param dropped features dropped for label leakage * @param featuresStatistics stats on features * @param names names of features passed in @@ -74,7 +74,7 @@ case object SanityCheckerNames extends DerivedFeatureFilterNames { */ case class SanityCheckerSummary ( - correlationsWLabel: Correlations, + correlations: Correlations, dropped: Seq[String], featuresStatistics: SummaryStatistics, names: Seq[String], @@ -92,9 +92,8 @@ case class SanityCheckerSummary sample: Double ) { this( - correlationsWLabel = new Correlations( - stats.filter(s => s.corrLabel.isDefined && !s.corrLabel.get.isNaN).map(s => s.name -> s.corrLabel.get), - stats.filter(s => s.corrLabel.isDefined && s.corrLabel.get.isNaN).map(_.name), + correlations = new Correlations( + stats.filter(s => s.corrLabel.isDefined).map(s => (s.name, s.corrLabel.get, s.featureCorrs)), correlationType ), dropped = dropped, @@ -113,7 +112,7 @@ case class SanityCheckerSummary */ def toMetadata(skipUnsupported: Boolean): Metadata = { val summaryMeta = new MetadataBuilder() - summaryMeta.putMetadata(SanityCheckerNames.CorrelationsWLabel, correlationsWLabel.toMetadata(skipUnsupported)) + summaryMeta.putMetadata(SanityCheckerNames.Correlations, correlations.toMetadata(skipUnsupported)) summaryMeta.putStringArray(SanityCheckerNames.Dropped, dropped.toArray) summaryMeta.putMetadata(SanityCheckerNames.FeaturesStatistics, featuresStatistics.toMetadata(skipUnsupported)) summaryMeta.putStringArray(SanityCheckerNames.Names, names.toArray) @@ -274,23 +273,24 @@ case class CategoricalStats * Correlations between features and the label from [[SanityChecker]] * * @param featuresIn names of features - * @param values correlation of feature with label - * @param nanCorrs nan correlation features + * @param valuesWithLabel correlation of feature with label + * @param valuesWithFeatures correlations between features features * @param corrType type of correlation done on */ case class Correlations ( featuresIn: Seq[String], - values: Seq[Double], - nanCorrs: Seq[String], + valuesWithLabel: Seq[Double], + valuesWithFeatures: Seq[Seq[Double]], corrType: CorrelationType ) extends MetadataLike { - require(featuresIn.length == values.length, "Feature names and correlation values arrays must have the same length") + require(featuresIn.length == valuesWithLabel.length, + "Feature names and correlation values arrays must have the same length") - def this(corrs: Seq[(String, Double)], nans: Seq[String], corrType: CorrelationType) = this( + def this(corrs: Seq[(String, Double, Seq[Double])], corrType: CorrelationType) = this( featuresIn = corrs.map(_._1), - values = corrs.map(_._2), - nanCorrs = nans, + valuesWithLabel = corrs.map(_._2), + valuesWithFeatures = corrs.map(_._3), corrType = corrType ) @@ -303,8 +303,12 @@ case class Correlations def toMetadata(skipUnsupported: Boolean): Metadata = { val corrMeta = new MetadataBuilder() corrMeta.putStringArray(SanityCheckerNames.FeaturesIn, featuresIn.toArray) - corrMeta.putDoubleArray(SanityCheckerNames.Values, values.toArray) - corrMeta.putStringArray(SanityCheckerNames.CorrelationsWLabelIsNaN, nanCorrs.toArray) + corrMeta.putStringArray(SanityCheckerNames.ValuesLabel, valuesWithLabel.map(_.toString).toArray) + val fcMeta = new MetadataBuilder + if (valuesWithFeatures.nonEmpty) { + valuesWithFeatures.zip(featuresIn).map(c => fcMeta.putStringArray(c._2, c._1.map(_.toString).toArray)) + } + corrMeta.putMetadata(SanityCheckerNames.ValuesFeatures, fcMeta.build()) corrMeta.putString(SanityCheckerNames.CorrelationType, corrType.sparkName) corrMeta.build() } @@ -319,27 +323,32 @@ case class Correlations } else { corrType } - new Correlations(featuresIn ++ corr.featuresIn, values ++ corr.values, nanCorrs ++ corr.nanCorrs, corrName) + new Correlations(featuresIn ++ corr.featuresIn, valuesWithLabel ++ corr.valuesWithLabel, + valuesWithFeatures ++ corr.valuesWithFeatures, corrName) } } case object SanityCheckerSummary { def flatten(checkers: Seq[SanityCheckerSummary]): SanityCheckerSummary = { - val correlationsWLabel: Correlations = checkers.map(_.correlationsWLabel).reduce(_ + _) + val correlations: Correlations = checkers.map(_.correlations).reduce(_ + _) val dropped: Seq[String] = checkers.flatMap(_.dropped) val featuresStatistics: SummaryStatistics = checkers.map(_.featuresStatistics).reduce(_ + _) val names: Seq[String] = checkers.flatMap(_.names) val categoricalStats: Array[CategoricalGroupStats] = checkers.flatMap(_.categoricalStats).toArray - new SanityCheckerSummary(correlationsWLabel, dropped, featuresStatistics, names, categoricalStats) + new SanityCheckerSummary(correlations, dropped, featuresStatistics, names, categoricalStats) } private def correlationsFromMetadata(meta: Metadata): Correlations = { val wrapped = meta.wrapped + val features = wrapped.getArray[String](SanityCheckerNames.FeaturesIn).toSeq + val fc = wrapped.get[Metadata](SanityCheckerNames.ValuesFeatures).wrapped Correlations( featuresIn = wrapped.getArray[String](SanityCheckerNames.FeaturesIn).toSeq, - values = wrapped.getArray[Double](SanityCheckerNames.Values).toSeq, - nanCorrs = wrapped.getArray[String](SanityCheckerNames.CorrelationsWLabelIsNaN).toSeq, + valuesWithLabel = wrapped.getArray[String](SanityCheckerNames.ValuesLabel).toSeq.map(_.toDouble), + valuesWithFeatures = + if (fc.underlyingMap.isEmpty) Seq.empty + else features.map(f => fc.getArray[String](f).toSeq.map(_.toDouble)), corrType = CorrelationType.withNameInsensitive(wrapped.get[String](SanityCheckerNames.CorrelationType)) ) } @@ -383,7 +392,7 @@ case object SanityCheckerSummary { // Try parsing as an older version of metadata (pre-3.3.0) if this doesn't work Try { SanityCheckerSummary( - correlationsWLabel = correlationsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.CorrelationsWLabel)), + correlations = correlationsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.Correlations)), dropped = wrapped.getArray[String](SanityCheckerNames.Dropped).toSeq, featuresStatistics = statisticsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.FeaturesStatistics)), names = wrapped.getArray[String](SanityCheckerNames.Names).toSeq, diff --git a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala index 7412ba5c7d..1a7a88a7fa 100644 --- a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala +++ b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala @@ -526,7 +526,7 @@ class ModelInsightsTest extends FlatSpec with PassengerSparkFixtureTest with Dou val labelName = "l" val summary = SanityCheckerSummary( - correlationsWLabel = Correlations(Seq("f0_f0_f2_1", "f0_f0_f3_2"), Seq(5.2, 5.3), Seq("f1_0"), + correlations = Correlations(Seq("f0_f0_f2_1", "f0_f0_f3_2"), Seq(5.2, 5.3), Seq.empty, CorrelationType.Pearson), dropped = Seq("f1_0"), featuresStatistics = SummaryStatistics(count = 3, sampleFraction = 0.01, max = Seq(0.1, 0.2, 0.3, 0.0), diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala index bf3d025626..e90f021b7b 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala @@ -43,7 +43,8 @@ import org.scalatest.junit.JUnitRunner class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { val summary = SanityCheckerSummary( - correlationsWLabel = Correlations(Seq("f2", "f3"), Seq(0.2, 0.3), Seq(), CorrelationType.Pearson), + correlations = Correlations(Seq("f2", "f3"), Seq(0.2, 0.3), Seq(Seq(0.2, 0.3), Seq(0.3, 0.2)), + CorrelationType.Pearson), dropped = Seq("f1"), featuresStatistics = SummaryStatistics(3, 0.01, Seq(0.1, 0.2, 0.3), Seq(0.1, 0.2, 0.3), Seq(0.1, 0.2, 0.3), Seq(0.1, 0.2, 0.3)), @@ -79,10 +80,10 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { val retrieved = SanityCheckerSummary.fromMetadata(meta) retrieved.isInstanceOf[SanityCheckerSummary] - retrieved.correlationsWLabel.nanCorrs should contain theSameElementsAs summary.correlationsWLabel.nanCorrs + retrieved.correlations.valuesWithFeatures should contain theSameElementsAs summary.correlations.valuesWithFeatures - retrieved.correlationsWLabel.featuresIn should contain theSameElementsAs summary.correlationsWLabel.featuresIn - retrieved.correlationsWLabel.values should contain theSameElementsAs summary.correlationsWLabel.values + retrieved.correlations.featuresIn should contain theSameElementsAs summary.correlations.featuresIn + retrieved.correlations.valuesWithLabel should contain theSameElementsAs summary.correlations.valuesWithLabel retrieved.categoricalStats.map(_.cramersV) should contain theSameElementsAs summary.categoricalStats.map(_.cramersV) @@ -95,7 +96,7 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { retrieved.featuresStatistics.variance should contain theSameElementsAs summary.featuresStatistics.variance retrieved.names should contain theSameElementsAs summary.names - retrieved.correlationsWLabel.corrType shouldBe summary.correlationsWLabel.corrType + retrieved.correlations.corrType shouldBe summary.correlations.corrType retrieved.categoricalStats.flatMap(_.categoricalFeatures) should contain theSameElementsAs summary.categoricalStats.flatMap(_.categoricalFeatures) diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index 70d7d420a2..ccab969cf0 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -158,7 +158,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val transformedData = model.transform(testData) val expectedFeatNames = expectedCorrFeatNames ++ expectedCorrFeatNamesIsNan - validateTransformerOutput(outputColName, transformedData, expectedFeatNames, expectedCorrFeatNames, + validateTransformerOutput(outputColName, transformedData, expectedFeatNames, featuresToDrop, expectedCorrFeatNamesIsNan) } @@ -209,7 +209,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP outputColumns.columns.length + summary.dropped.length shouldEqual testMetadata.columns.length val expectedFeatNames = expectedCorrFeatNames ++ expectedCorrFeatNamesIsNan - validateTransformerOutput(outputColName, transformedData, expectedFeatNames, expectedCorrFeatNames, + validateTransformerOutput(outputColName, transformedData, expectedFeatNames, featuresToDrop, expectedCorrFeatNamesIsNan) } @@ -228,7 +228,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val transformedData = model.transform(testData) val expectedFeatNames = expectedCorrFeatNames ++ expectedCorrFeatNamesIsNan - validateTransformerOutput(outputColName, transformedData, expectedFeatNames, expectedCorrFeatNames, + validateTransformerOutput(outputColName, transformedData, expectedFeatNames, Seq(), expectedCorrFeatNamesIsNan) } @@ -318,9 +318,9 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val summaryMeta = model.getMetadata().getSummaryMetadata() val correlations = summaryMeta - .getMetadata(SanityCheckerNames.CorrelationsWLabel) - .getDoubleArray(SanityCheckerNames.Values) - correlations(0) + .getMetadata(SanityCheckerNames.Correlations) + .getStringArray(SanityCheckerNames.ValuesLabel) + correlations(0).toDouble } val sanityCheckerSpearman = new SanityChecker() @@ -383,7 +383,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val featuresWithNaNCorr = Seq("textMap_7") val expectedFeatNames = featuresWithCorr ++ featuresWithNaNCorr - validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresWithCorr, + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresToDrop, featuresWithNaNCorr) } @@ -424,7 +424,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP ) val expectedFeatNames = featuresWithCorr ++ featuresWithNaNCorr - validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresWithCorr, + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresToDrop, featuresWithNaNCorr) } @@ -460,7 +460,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP ) val featuresWithNaNCorr = Seq.empty[String] - validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresWithCorr, + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresToDrop, featuresWithNaNCorr) } @@ -494,7 +494,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP ) val featuresWithNaNCorr = Seq.empty[String] - validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresWithCorr, + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresToDrop, featuresWithNaNCorr) } @@ -528,7 +528,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val featuresWithNaNCorr = Seq("textMap_7") val expectedFeatNames = featuresWithCorr ++ featuresWithNaNCorr - validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresWithCorr, + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresToDrop, featuresWithNaNCorr) } @@ -575,7 +575,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val expectedFeatNames = featuresWithCorr ++ featuresWithNaNCorr - validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresWithCorr, + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, featuresToDrop, featuresWithNaNCorr) } @@ -629,9 +629,9 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP outputColName: String, transformedData: DataFrame, expectedFeatNames: Seq[String], - expectedCorrFeatNames: Seq[String], expectedFeaturesToDrop: Seq[String], - expectedCorrFeatNamesIsNan: Seq[String] + expectedCorrFeatNamesIsNan: Seq[String], + ignoredNames: Seq[String] = Seq.empty ): Unit = { transformedData.select(outputColName).collect().foreach { case Row(features: Vector) => features.toArray.length equals @@ -643,8 +643,10 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP summary.names.slice(0, summary.names.size - 1) should contain theSameElementsAs expectedFeatNames - summary.correlationsWLabel.nanCorrs should contain theSameElementsAs expectedCorrFeatNamesIsNan - summary.correlationsWLabel.featuresIn should contain theSameElementsAs expectedCorrFeatNames + summary.correlations.valuesWithLabel.zip(summary.names).collect{ + case (corr, name) if corr.isNaN => name + } should contain theSameElementsAs expectedCorrFeatNamesIsNan + summary.correlations.featuresIn should contain theSameElementsAs expectedFeatNames.diff(ignoredNames) summary.dropped should contain theSameElementsAs expectedFeaturesToDrop } From b1dca9cb456980d60d389a28eb01f12090a4b8a7 Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Thu, 7 May 2020 15:46:57 -0700 Subject: [PATCH 2/9] fixed some tests --- .../scala/com/salesforce/op/ModelInsightsTest.scala | 2 +- .../preparators/SanityCheckerMetadataTest.scala | 13 ++++++++++--- .../stages/impl/preparators/SanityCheckerTest.scala | 13 +++++-------- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala index 1a7a88a7fa..57edd9f68b 100644 --- a/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala +++ b/core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala @@ -526,7 +526,7 @@ class ModelInsightsTest extends FlatSpec with PassengerSparkFixtureTest with Dou val labelName = "l" val summary = SanityCheckerSummary( - correlations = Correlations(Seq("f0_f0_f2_1", "f0_f0_f3_2"), Seq(5.2, 5.3), Seq.empty, + correlations = Correlations(Seq("f1_0", "f0_f0_f2_1", "f0_f0_f3_2"), Seq(Double.NaN, 5.2, 5.3), Seq.empty, CorrelationType.Pearson), dropped = Seq("f1_0"), featuresStatistics = SummaryStatistics(count = 3, sampleFraction = 0.01, max = Seq(0.1, 0.2, 0.3, 0.0), diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala index e90f021b7b..5dc6b90253 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala @@ -43,7 +43,8 @@ import org.scalatest.junit.JUnitRunner class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { val summary = SanityCheckerSummary( - correlations = Correlations(Seq("f2", "f3"), Seq(0.2, 0.3), Seq(Seq(0.2, 0.3), Seq(0.3, 0.2)), + correlations = Correlations(Seq("f2", "f3", "f4"), Seq(0.2, 0.3, Double.NaN), Seq(Seq(0.2, 0.3, 0.1), + Seq(0.3, 0.2, Double.NaN), Seq(0.1, 0.1, 0.1)), CorrelationType.Pearson), dropped = Seq("f1"), featuresStatistics = SummaryStatistics(3, 0.01, Seq(0.1, 0.2, 0.3), Seq(0.1, 0.2, 0.3), @@ -80,10 +81,16 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { val retrieved = SanityCheckerSummary.fromMetadata(meta) retrieved.isInstanceOf[SanityCheckerSummary] - retrieved.correlations.valuesWithFeatures should contain theSameElementsAs summary.correlations.valuesWithFeatures retrieved.correlations.featuresIn should contain theSameElementsAs summary.correlations.featuresIn - retrieved.correlations.valuesWithLabel should contain theSameElementsAs summary.correlations.valuesWithLabel + retrieved.correlations.valuesWithLabel.zip(summary.correlations.valuesWithLabel).foreach{ + case (rd, id) => if (rd.isNaN) id.isNaN shouldBe true else rd shouldEqual id + } + retrieved.correlations.valuesWithFeatures.zip(summary.correlations.valuesWithFeatures).foreach{ + case (rs, is) => rs.zip(is).foreach{ + case (rd, id) => if (rd.isNaN) id.isNaN shouldBe true else rd shouldEqual id + } + } retrieved.categoricalStats.map(_.cramersV) should contain theSameElementsAs summary.categoricalStats.map(_.cramersV) diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index ccab969cf0..a2cd644b6e 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -455,13 +455,12 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP "textMap_6", "textMap_7", "textMap_color_NullIndicatorValue_8", "textMap_fruit_NullIndicatorValue_9", "textMap_beverage_NullIndicatorValue_10" ) - val featuresWithCorr = Seq("textMap_color_NullIndicatorValue_8", "textMap_fruit_NullIndicatorValue_9", - "textMap_beverage_NullIndicatorValue_10" - ) + val featuresIgnore = Seq("textMap_0", "textMap_1", "textMap_2", "textMap_3", "textMap_4", "textMap_5", + "textMap_6", "textMap_7") val featuresWithNaNCorr = Seq.empty[String] validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, - featuresToDrop, featuresWithNaNCorr) + featuresToDrop, featuresWithNaNCorr, featuresIgnore) } it should "not calculate correlations on hashed text features if asked not to (using vectorizer)" in { @@ -489,13 +488,11 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val expectedFeatNames = (0 until 512).map(i => "textMap_" + i.toString) ++ Seq("textMap_beverage_NullIndicatorValue_512", "textMap_color_NullIndicatorValue_513", "textMap_fruit_NullIndicatorValue_514") - val featuresWithCorr = Seq("textMap_beverage_NullIndicatorValue_512", "textMap_color_NullIndicatorValue_513", - "textMap_fruit_NullIndicatorValue_514" - ) + val featuresIngore = (0 until 512).map(i => "textMap_" + i.toString) val featuresWithNaNCorr = Seq.empty[String] validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, - featuresToDrop, featuresWithNaNCorr) + featuresToDrop, featuresWithNaNCorr, featuresIngore) } it should "only calculate correlations between feature and the label if requested" in { From 0449f48d50ceeab479193bafbcc97bb7fda6e46f Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Fri, 8 May 2020 14:26:32 -0700 Subject: [PATCH 3/9] fixed more tests --- .../op/dsl/RichNumericFeature.scala | 2 ++ .../DerivedFeatureFilterUtils.scala | 2 +- .../impl/preparators/SanityChecker.scala | 12 +++---- .../preparators/SanityCheckerMetadata.scala | 31 +++++++++++++------ .../impl/preparators/BadFeatureZooTest.scala | 6 ++++ .../SanityCheckerMetadataTest.scala | 7 +++++ .../impl/preparators/SanityCheckerTest.scala | 4 +++ 7 files changed, 47 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala b/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala index c4cfa6f224..04a39e8d55 100644 --- a/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala +++ b/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala @@ -474,6 +474,7 @@ trait RichNumericFeature { sampleUpperLimit: Int = SanityChecker.SampleUpperLimit, maxCorrelation: Double = SanityChecker.MaxCorrelation, minCorrelation: Double = SanityChecker.MinCorrelation, + maxFeatureCorr: Double = SanityChecker.MaxFeatureCorr, maxCramersV: Double = SanityChecker.MaxCramersV, correlationType: CorrelationType = SanityChecker.CorrelationTypeDefault, minVariance: Double = SanityChecker.MinVariance, @@ -494,6 +495,7 @@ trait RichNumericFeature { .setSampleUpperLimit(sampleUpperLimit) .setMaxCorrelation(maxCorrelation) .setMinCorrelation(minCorrelation) + .setMaxFeatureCorr(maxFeatureCorr) .setMaxCramersV(maxCramersV) .setCorrelationType(correlationType) .setMinVariance(minVariance) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala index c4e0234d93..4a4e5b8291 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala @@ -178,7 +178,7 @@ object DerivedFeatureFilterUtils { cramersV = cramersVMap.get(name), featureCorrs = corrIndices.indexOf(i) match { case -1 => Seq.empty - case ind => featureCorrs.getOrElse(Array.empty).map(_.apply(ind)) + case ind => featureCorrs.getOrElse(Array.empty).map(_.apply(ind)).dropRight(1) // drop label corr }, parentCorr = getParentValue(col, corrParent, corrParentNoKeys), parentCramersV = getParentValue(col, cramersVParent, cramersVParentNoKeys), diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala index ffbc48e220..c7d78fdd6e 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala @@ -94,7 +94,7 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { final val maxCorrelation = new DoubleParam( parent = this, name = "maxCorrelation", doc = "Maximum correlation (absolute value) allowed between a feature in the feature vector and the label", - isValid = ParamValidators.inRange(lowerBound = 0.0, upperBound = 1.0, lowerInclusive = true, upperInclusive = true) + isValid = ParamValidators.inRange(lowerBound = -0.1, upperBound = 1.1, lowerInclusive = true, upperInclusive = true) ) def setMaxCorrelation(value: Double): this.type = set(maxCorrelation, value) def getMaxCorrelation: Double = $(maxCorrelation) @@ -103,15 +103,15 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { parent = this, name = "maxFeatureCorr", doc = "Maximum correlation (absolute value) allowed between a feature two feature vectors which will" + " both be kept. When this value is exceeded the second feature in the correlated pair will be dropped", - isValid = ParamValidators.inRange(lowerBound = 0.0, upperBound = 1.0, lowerInclusive = true, upperInclusive = true) + isValid = ParamValidators.inRange(lowerBound = -0.1, upperBound = 1.1, lowerInclusive = true, upperInclusive = true) ) - def setMaxFeatureCorr(value: Double): this.type = set(maxCorrelation, value) - def getMaxFeatureCorr: Double = $(maxCorrelation) + def setMaxFeatureCorr(value: Double): this.type = set(maxFeatureCorr, value) + def getMaxFeatureCorr: Double = $(maxFeatureCorr) final val minCorrelation = new DoubleParam( parent = this, name = "minCorrelation", doc = "Minimum correlation (absolute value) allowed between a feature in the feature vector and the label", - isValid = ParamValidators.inRange(lowerBound = 0.0, upperBound = 1.0, lowerInclusive = true, upperInclusive = true) + isValid = ParamValidators.inRange(lowerBound = -0.1, upperBound = 1.1, lowerInclusive = true, upperInclusive = true) ) def setMinCorrelation(value: Double): this.type = set(minCorrelation, value) def getMinCorrelation: Double = $(minCorrelation) @@ -554,7 +554,7 @@ object SanityChecker { val SampleLowerLimit = 1E3.toInt val SampleUpperLimit = 1E6.toInt val MaxCorrelation = 0.95 - val MaxFeatureCorr = 0.999 + val MaxFeatureCorr = 0.99 val MinCorrelation = 0.0 val MinVariance = 1E-5 val MaxCramersV = 0.95 diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala index e2b9753805..84e5389050 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala @@ -323,7 +323,7 @@ case class Correlations } else { corrType } - new Correlations(featuresIn ++ corr.featuresIn, valuesWithLabel ++ corr.valuesWithLabel, + Correlations(featuresIn ++ corr.featuresIn, valuesWithLabel ++ corr.valuesWithLabel, valuesWithFeatures ++ corr.valuesWithFeatures, corrName) } } @@ -342,15 +342,26 @@ case object SanityCheckerSummary { private def correlationsFromMetadata(meta: Metadata): Correlations = { val wrapped = meta.wrapped val features = wrapped.getArray[String](SanityCheckerNames.FeaturesIn).toSeq - val fc = wrapped.get[Metadata](SanityCheckerNames.ValuesFeatures).wrapped - Correlations( - featuresIn = wrapped.getArray[String](SanityCheckerNames.FeaturesIn).toSeq, - valuesWithLabel = wrapped.getArray[String](SanityCheckerNames.ValuesLabel).toSeq.map(_.toDouble), - valuesWithFeatures = - if (fc.underlyingMap.isEmpty) Seq.empty - else features.map(f => fc.getArray[String](f).toSeq.map(_.toDouble)), - corrType = CorrelationType.withNameInsensitive(wrapped.get[String](SanityCheckerNames.CorrelationType)) - ) + if (wrapped.underlyingMap.keySet.contains("values")) { // old sanity checker meta + val nans = wrapped.getArray[String]("correlationsWithLabelIsNaN") + val labelCorr = wrapped.getArray[Double]("values").toSeq + Correlations( + featuresIn = features ++ nans, + valuesWithLabel = labelCorr ++ Seq.fill(nans.length)(Double.NaN), + valuesWithFeatures = Seq.empty, + corrType = CorrelationType.withNameInsensitive(wrapped.get[String](SanityCheckerNames.CorrelationType)) + ) + } else { + val fc = wrapped.get[Metadata](SanityCheckerNames.ValuesFeatures).wrapped + Correlations( + featuresIn = features, + valuesWithLabel = wrapped.getArray[String](SanityCheckerNames.ValuesLabel).toSeq.map(_.toDouble), + valuesWithFeatures = + if (fc.underlyingMap.isEmpty) Seq.empty + else features.map(f => fc.getArray[String](f).toSeq.map(_.toDouble)), + corrType = CorrelationType.withNameInsensitive(wrapped.get[String](SanityCheckerNames.CorrelationType)) + ) + } } private def statisticsFromMetadata(meta: Metadata): SummaryStatistics = { diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala index 01256afc49..5420f48413 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala @@ -88,6 +88,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawCurrency).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) + .setMaxFeatureCorr(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() @@ -98,6 +99,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { // Check that all the PickList features are dropped and they are the only ones dropped // (9 choices + "other" + empty = 11 total dropped columns) + println(retrieved.dropped) retrieved.dropped.forall(_.startsWith("picklist")) shouldBe true retrieved.dropped.length shouldBe 11 retrieved.categoricalStats.flatMap(_.categoricalFeatures).length shouldBe @@ -245,6 +247,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawCurrency).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) + .setMaxFeatureCorr(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() @@ -289,6 +292,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawCurrency).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) + .setMaxFeatureCorr(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .setCategoricalLabel(false) @@ -333,6 +337,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawIntegralMap).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) + .setMaxFeatureCorr(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() @@ -642,6 +647,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) + .setMaxFeatureCorr(1.1) .setInput(labels, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala index 5dc6b90253..baf2fdcdea 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala @@ -120,4 +120,11 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { recovered.hashCode() shouldEqual summary.toMetadata().hashCode() } + it should "be able to read metadata from the old format" in { + val json = "" + val recovered = Metadata.fromJson(json) + val summaryRecovered = SanityCheckerSummary.fromMetadata(recovered) + summaryRecovered.correlations.valuesWithLabel.filter(_.isNaN) shouldBe true + } + } diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index a2cd644b6e..7a3e94ad72 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -183,6 +183,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP checker.getOrDefault(checker.maxCorrelation) shouldBe SanityChecker.MaxCorrelation checker.getOrDefault(checker.minVariance) shouldBe SanityChecker.MinVariance checker.getOrDefault(checker.minCorrelation) shouldBe SanityChecker.MinCorrelation + checker.getOrDefault(checker.maxFeatureCorr) shouldBe SanityChecker.MaxFeatureCorr checker.getOrDefault(checker.correlationType) shouldBe CorrelationType.Pearson.entryName checker.getOrDefault(checker.removeBadFeatures) shouldBe SanityChecker.RemoveBadFeatures } @@ -428,6 +429,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP featuresToDrop, featuresWithNaNCorr) } + it should "not calculate correlations on hashed text features if asked not to (using SmartTextVectorizer)" in { val smartMapVectorized = new SmartTextMapVectorizer[TextMap]() .setMaxCardinality(2).setNumFeatures(8).setMinSupport(1).setTopK(2).setPrependFeatureName(true) @@ -442,6 +444,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP .setCorrelationExclusion(CorrelationExclusion.HashedText) .setMinCorrelation(0.0) .setMaxCorrelation(0.8) + .setMaxFeatureCorr(1.0) .setMaxCramersV(0.8) .setInput(targetResponse, smartMapVectorized) .getOutput() @@ -476,6 +479,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP .setMinVariance(-0.1) .setMinCorrelation(0.0) .setMaxCorrelation(0.8) + .setMaxFeatureCorr(1.0) .setMaxCramersV(0.8) .setInput(targetResponse, vectorized) .getOutput() From ce3554517b1711d03c49a0e7638da75164cdf332 Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Fri, 8 May 2020 15:15:55 -0700 Subject: [PATCH 4/9] added test --- .../impl/preparators/SanityCheckerTest.scala | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index 7a3e94ad72..7689219da6 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -31,6 +31,7 @@ package com.salesforce.op.stages.impl.preparators import com.salesforce.op._ + import com.salesforce.op.features.FeatureLike import com.salesforce.op.features.types._ import com.salesforce.op.stages.MetadataParam @@ -466,6 +467,35 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP featuresToDrop, featuresWithNaNCorr, featuresIgnore) } + it should "not through out duplicate features if their correlation is above the max feature corr" in { + + val checkedFeatures = new SanityChecker() + .setCheckSample(1.0) + .setRemoveBadFeatures(true) + .setRemoveFeatureGroup(false) + .setProtectTextSharedHash(false) + .setCorrelationExclusion(CorrelationExclusion.HashedText) + .setMinVariance(-0.1) + .setMinCorrelation(-0.1) + .setMaxCorrelation(1.1) + .setMaxFeatureCorr(0.9) + .setMaxCramersV(1.0) + .setInput(targetLabel, featureVector) + .getOutput() + + checkedFeatures.originStage shouldBe a[SanityChecker] + + val transformed = new OpWorkflow().setResultFeatures(featureVector, checkedFeatures).transform(testData) + + val featuresToDrop = Seq("testFeatNegCor_4") + val expectedFeatNames = Seq("age_0", "height_1", "height_null_2", "gender_3", "testFeatNegCor_4") + val featuresIngore = Seq.empty + val featuresWithNaNCorr = Seq("age_0") + + validateTransformerOutput(checkedFeatures.name, transformed, expectedFeatNames, + featuresToDrop, featuresWithNaNCorr, featuresIngore) + } + it should "not calculate correlations on hashed text features if asked not to (using vectorizer)" in { val vectorized = textMap.vectorize(cleanText = TransmogrifierDefaults.CleanText) @@ -641,7 +671,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val metadata: Metadata = getMetadata(outputColName, transformedData) val summary = SanityCheckerSummary.fromMetadata(metadata.getSummaryMetadata()) - +println(summary.correlations) summary.names.slice(0, summary.names.size - 1) should contain theSameElementsAs expectedFeatNames summary.correlations.valuesWithLabel.zip(summary.names).collect{ From 85f2159d43d99fdd967c2c5b0a65637c176e152b Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Fri, 8 May 2020 15:43:02 -0700 Subject: [PATCH 5/9] added metadata test --- .../preparators/SanityCheckerMetadata.scala | 15 +++-- .../SanityCheckerMetadataTest.scala | 57 ++++++++++++++++++- 2 files changed, 65 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala index 84e5389050..f23a8842b7 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala @@ -402,18 +402,23 @@ case object SanityCheckerSummary { val wrapped = meta.wrapped // Try parsing as an older version of metadata (pre-3.3.0) if this doesn't work Try { + val corr = + if (wrapped.underlyingMap.contains("correlationsWithLabel")) { + wrapped.get[Metadata]("correlationsWithLabel") + } else wrapped.get[Metadata](SanityCheckerNames.Correlations) SanityCheckerSummary( - correlations = correlationsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.Correlations)), + correlations = correlationsFromMetadata(corr), dropped = wrapped.getArray[String](SanityCheckerNames.Dropped).toSeq, featuresStatistics = statisticsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.FeaturesStatistics)), names = wrapped.getArray[String](SanityCheckerNames.Names).toSeq, categoricalStats = wrapped.getArray[Metadata](SanityCheckerNames.CategoricalStats) .map(categoricalGroupStatsFromMetadata) ) - } match { - case Success(summary) => summary - // Parse it under the old format - case Failure(_) => throw new IllegalArgumentException(s"failed to parse SanityCheckerSummary from $meta") } + } match { + case Success(summary) => summary + // Parse it under the old format + case Failure(_) => throw new IllegalArgumentException(s"failed to parse SanityCheckerSummary from $meta") } } + diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala index baf2fdcdea..c05aef0796 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala @@ -121,10 +121,63 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { } it should "be able to read metadata from the old format" in { - val json = "" + val json = """{ + | "statistics" : { + | "sampleFraction" : 0.01, + | "count" : 3.0, + | "variance" : [ 0.1, 0.2, 0.3 ], + | "mean" : [ 0.1, 0.2, 0.3 ], + | "min" : [ 0.1, 0.2, 0.3 ], + | "max" : [ 0.1, 0.2, 0.3 ] + | }, + | "names" : [ "f1", "f2", "f3" ], + | "correlationsWithLabel" : { + | "correlationType" : "pearson", + | "values" : [ 0.2, 0.3 ], + | "features" : [ "f2", "f3" ], + | "correlationsWithLabelIsNaN" : [ ] + | }, + | "categoricalStats" : [ { + | "support" : [ 1.0 ], + | "contingencyMatrix" : { + | "2" : [ 12.0 ], + | "1" : [ 12.0 ], + | "0" : [ 12.0 ] + | }, + | "maxRuleConfidence" : [ 1.0 ], + | "categoricalFeatures" : [ "f4" ], + | "pointwiseMutualInfoAgainstLabel" : { + | "2" : [ -0.32 ], + | "1" : [ 1.11 ], + | "0" : [ 1.23 ] + | }, + | "mutualInfo" : -1.22, + | "cramersV" : 0.45, + | "group" : "f4" + | }, { + | "support" : [ 1.0 ], + | "contingencyMatrix" : { + | "2" : [ 12.0 ], + | "1" : [ 12.0 ], + | "0" : [ 12.0 ] + | }, + | "maxRuleConfidence" : [ 1.0 ], + | "categoricalFeatures" : [ "f5" ], + | "pointwiseMutualInfoAgainstLabel" : { + | "2" : [ 0.99 ], + | "1" : [ 0.34 ], + | "0" : [ -2.11 ] + | }, + | "mutualInfo" : -0.51, + | "cramersV" : 0.11, + | "group" : "f5" + | } ], + | "featuresDropped" : [ "f1" ] + |}""".stripMargin val recovered = Metadata.fromJson(json) val summaryRecovered = SanityCheckerSummary.fromMetadata(recovered) - summaryRecovered.correlations.valuesWithLabel.filter(_.isNaN) shouldBe true + summaryRecovered.correlations.valuesWithLabel.size shouldBe 2 + summaryRecovered.correlations.valuesWithFeatures shouldBe Seq.empty } } From 04f10b60f43dec65fbdcf648107d02e8b2a57d6b Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Thu, 14 May 2020 10:20:58 -0700 Subject: [PATCH 6/9] addressing comments --- .../stages/impl/preparators/DerivedFeatureFilterUtils.scala | 2 +- .../salesforce/op/stages/impl/preparators/SanityChecker.scala | 4 ++-- .../op/stages/impl/preparators/SanityCheckerMetadata.scala | 4 +--- .../op/stages/impl/preparators/BadFeatureZooTest.scala | 1 - .../stages/impl/preparators/SanityCheckerMetadataTest.scala | 4 ++-- .../op/stages/impl/preparators/SanityCheckerTest.scala | 3 +-- 6 files changed, 7 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala index 4a4e5b8291..cf6aabfc02 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/DerivedFeatureFilterUtils.scala @@ -374,7 +374,7 @@ private[op] case class ColumnStatistics s"correlation $corr higher than max correlation $maxCorrelation" ), column.flatMap{ case cl => featureCorrs.take(cl.index).find(Math.abs(_) > maxFeatureCorr).map(corr => - s"this feature has correlations $corr with other features higher than max feature-feature" + + s"this feature has correlations $corr with another feature higher than max feature-feature" + s" correlation $maxFeatureCorr") }, cramersV.filter(_ > maxCramersV).map(cv => diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala index c7d78fdd6e..7b80a77c47 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala @@ -101,8 +101,8 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { final val maxFeatureCorr = new DoubleParam( parent = this, name = "maxFeatureCorr", - doc = "Maximum correlation (absolute value) allowed between a feature two feature vectors which will" + - " both be kept. When this value is exceeded the second feature in the correlated pair will be dropped", + doc = "Maximum correlation (absolute value) allowed between two features. When this value is exceeded the second" + + " feature in the correlated pair will be dropped.", isValid = ParamValidators.inRange(lowerBound = -0.1, upperBound = 1.1, lowerInclusive = true, upperInclusive = true) ) def setMaxFeatureCorr(value: Double): this.type = set(maxFeatureCorr, value) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala index f23a8842b7..cfe9fbe6fc 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala @@ -253,11 +253,9 @@ case class CategoricalStats * @throws RuntimeException in case of unsupported value type * @return [[Metadata]] metadata */ - // TODO: Build the metadata here instead of by treating Cramer's V and mutual info as correlations def toMetadata(skipUnsupported: Boolean): Metadata = { val meta = new MetadataBuilder() meta.putStringArray(SanityCheckerNames.CategoricalFeatures, categoricalFeatures) - // TODO: use custom serializer here instead of replacing NaNs with 0s meta.putDoubleArray(SanityCheckerNames.CramersV, cramersVs.map(f => if (f.isNaN) 0 else f)) meta.putDoubleArray(SanityCheckerNames.MutualInfo, mutualInfos) meta.putMetadata(SanityCheckerNames.PointwiseMutualInfoAgainstLabel, @@ -342,7 +340,7 @@ case object SanityCheckerSummary { private def correlationsFromMetadata(meta: Metadata): Correlations = { val wrapped = meta.wrapped val features = wrapped.getArray[String](SanityCheckerNames.FeaturesIn).toSeq - if (wrapped.underlyingMap.keySet.contains("values")) { // old sanity checker meta + if (wrapped.underlyingMap.keySet.contains("correlationsWithLabelIsNaN")) { // old sanity checker meta val nans = wrapped.getArray[String]("correlationsWithLabelIsNaN") val labelCorr = wrapped.getArray[Double]("values").toSeq Correlations( diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala index 5420f48413..688bb5761f 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala @@ -99,7 +99,6 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { // Check that all the PickList features are dropped and they are the only ones dropped // (9 choices + "other" + empty = 11 total dropped columns) - println(retrieved.dropped) retrieved.dropped.forall(_.startsWith("picklist")) shouldBe true retrieved.dropped.length shouldBe 11 retrieved.categoricalStats.flatMap(_.categoricalFeatures).length shouldBe diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala index c05aef0796..7e2fc275d8 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadataTest.scala @@ -135,7 +135,7 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { | "correlationType" : "pearson", | "values" : [ 0.2, 0.3 ], | "features" : [ "f2", "f3" ], - | "correlationsWithLabelIsNaN" : [ ] + | "correlationsWithLabelIsNaN" : ["f1"] | }, | "categoricalStats" : [ { | "support" : [ 1.0 ], @@ -176,7 +176,7 @@ class SanityCheckerMetadataTest extends FlatSpec with TestSparkContext { |}""".stripMargin val recovered = Metadata.fromJson(json) val summaryRecovered = SanityCheckerSummary.fromMetadata(recovered) - summaryRecovered.correlations.valuesWithLabel.size shouldBe 2 + summaryRecovered.correlations.valuesWithLabel.size shouldBe 3 summaryRecovered.correlations.valuesWithFeatures shouldBe Seq.empty } diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index 7689219da6..4d15350868 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -467,7 +467,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP featuresToDrop, featuresWithNaNCorr, featuresIgnore) } - it should "not through out duplicate features if their correlation is above the max feature corr" in { + it should "throw out duplicate features if their correlation is above the max feature corr" in { val checkedFeatures = new SanityChecker() .setCheckSample(1.0) @@ -671,7 +671,6 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP val metadata: Metadata = getMetadata(outputColName, transformedData) val summary = SanityCheckerSummary.fromMetadata(metadata.getSummaryMetadata()) -println(summary.correlations) summary.names.slice(0, summary.names.size - 1) should contain theSameElementsAs expectedFeatNames summary.correlations.valuesWithLabel.zip(summary.names).collect{ From fffbfdc43d3fd7b8acec895335d5fd532b63992e Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Mon, 18 May 2020 16:17:31 -0700 Subject: [PATCH 7/9] fixed typo --- .../op/stages/impl/preparators/SanityCheckerMetadata.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala index cfe9fbe6fc..c5713d0e8b 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerMetadata.scala @@ -272,7 +272,7 @@ case class CategoricalStats * * @param featuresIn names of features * @param valuesWithLabel correlation of feature with label - * @param valuesWithFeatures correlations between features features + * @param valuesWithFeatures correlations between features * @param corrType type of correlation done on */ case class Correlations From c86f8afb4b4edd567f434b3bbe0b44cc3fe0a374 Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Tue, 19 May 2020 11:10:23 -0700 Subject: [PATCH 8/9] changing param name --- .../op/stages/impl/preparators/SanityChecker.scala | 10 +++++----- .../op/stages/impl/preparators/SanityCheckerTest.scala | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala index 7b80a77c47..696a2f0659 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala @@ -99,14 +99,14 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { def setMaxCorrelation(value: Double): this.type = set(maxCorrelation, value) def getMaxCorrelation: Double = $(maxCorrelation) - final val maxFeatureCorr = new DoubleParam( + final val maxFeatureCorrelation = new DoubleParam( parent = this, name = "maxFeatureCorr", doc = "Maximum correlation (absolute value) allowed between two features. When this value is exceeded the second" + " feature in the correlated pair will be dropped.", isValid = ParamValidators.inRange(lowerBound = -0.1, upperBound = 1.1, lowerInclusive = true, upperInclusive = true) ) - def setMaxFeatureCorr(value: Double): this.type = set(maxFeatureCorr, value) - def getMaxFeatureCorr: Double = $(maxFeatureCorr) + def setMaxFeatureCorr(value: Double): this.type = set(maxFeatureCorrelation, value) + def getMaxFeatureCorr: Double = $(maxFeatureCorrelation) final val minCorrelation = new DoubleParam( parent = this, name = "minCorrelation", @@ -199,7 +199,7 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { sampleUpperLimit -> SanityChecker.SampleUpperLimit, maxCorrelation -> SanityChecker.MaxCorrelation, minCorrelation -> SanityChecker.MinCorrelation, - maxFeatureCorr -> SanityChecker.MaxFeatureCorr, + maxFeatureCorrelation -> SanityChecker.MaxFeatureCorr, minVariance -> SanityChecker.MinVariance, maxCramersV -> SanityChecker.MaxCramersV, removeBadFeatures -> SanityChecker.RemoveBadFeatures, @@ -489,7 +489,7 @@ class SanityChecker(uid: String = UID[SanityChecker]) $(minVariance), $(minCorrelation), $(maxCorrelation), - $(maxFeatureCorr), + $(maxFeatureCorrelation), $(maxCramersV), $(maxRuleConfidence), $(minRequiredRuleSupport), diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index fa47e556c9..cd19016106 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -184,7 +184,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP checker.getOrDefault(checker.maxCorrelation) shouldBe SanityChecker.MaxCorrelation checker.getOrDefault(checker.minVariance) shouldBe SanityChecker.MinVariance checker.getOrDefault(checker.minCorrelation) shouldBe SanityChecker.MinCorrelation - checker.getOrDefault(checker.maxFeatureCorr) shouldBe SanityChecker.MaxFeatureCorr + checker.getOrDefault(checker.maxFeatureCorrelation) shouldBe SanityChecker.MaxFeatureCorr checker.getOrDefault(checker.correlationType) shouldBe CorrelationType.Pearson.entryName checker.getOrDefault(checker.removeBadFeatures) shouldBe SanityChecker.RemoveBadFeatures } From ac1c3a7b453112069f1f7c185aa8d692db04dd66 Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Tue, 19 May 2020 11:14:49 -0700 Subject: [PATCH 9/9] chaging name --- .../com/salesforce/op/dsl/RichNumericFeature.scala | 2 +- .../op/stages/impl/preparators/SanityChecker.scala | 4 ++-- .../op/stages/impl/preparators/BadFeatureZooTest.scala | 10 +++++----- .../op/stages/impl/preparators/SanityCheckerTest.scala | 6 +++--- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala b/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala index 04a39e8d55..3f5448a1df 100644 --- a/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala +++ b/core/src/main/scala/com/salesforce/op/dsl/RichNumericFeature.scala @@ -495,7 +495,7 @@ trait RichNumericFeature { .setSampleUpperLimit(sampleUpperLimit) .setMaxCorrelation(maxCorrelation) .setMinCorrelation(minCorrelation) - .setMaxFeatureCorr(maxFeatureCorr) + .setMaxFeatureCorrelation(maxFeatureCorr) .setMaxCramersV(maxCramersV) .setCorrelationType(correlationType) .setMinVariance(minVariance) diff --git a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala index 696a2f0659..ab436a05a0 100644 --- a/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala +++ b/core/src/main/scala/com/salesforce/op/stages/impl/preparators/SanityChecker.scala @@ -105,8 +105,8 @@ trait SanityCheckerParams extends DerivedFeatureFilterParams { " feature in the correlated pair will be dropped.", isValid = ParamValidators.inRange(lowerBound = -0.1, upperBound = 1.1, lowerInclusive = true, upperInclusive = true) ) - def setMaxFeatureCorr(value: Double): this.type = set(maxFeatureCorrelation, value) - def getMaxFeatureCorr: Double = $(maxFeatureCorrelation) + def setMaxFeatureCorrelation(value: Double): this.type = set(maxFeatureCorrelation, value) + def getMaxFeatureCorrelation: Double = $(maxFeatureCorrelation) final val minCorrelation = new DoubleParam( parent = this, name = "minCorrelation", diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala index 688bb5761f..1cec33d518 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/BadFeatureZooTest.scala @@ -88,7 +88,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawCurrency).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) - .setMaxFeatureCorr(1.1) + .setMaxFeatureCorrelation(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() @@ -246,7 +246,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawCurrency).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) - .setMaxFeatureCorr(1.1) + .setMaxFeatureCorrelation(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() @@ -291,7 +291,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawCurrency).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) - .setMaxFeatureCorr(1.1) + .setMaxFeatureCorrelation(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .setCategoricalLabel(false) @@ -336,7 +336,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList, rawIntegralMap).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) - .setMaxFeatureCorr(1.1) + .setMaxFeatureCorrelation(1.1) .setInput(labelData, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() @@ -646,7 +646,7 @@ class BadFeatureZooTest extends FlatSpec with TestSparkContext with Logging { val genFeatureVector = Seq(rawCity, rawCountry, rawPickList).transmogrify() val checkedFeatures = new SanityChecker() .setCheckSample(1.0) - .setMaxFeatureCorr(1.1) + .setMaxFeatureCorrelation(1.1) .setInput(labels, genFeatureVector) .setRemoveBadFeatures(true) .getOutput() diff --git a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala index cd19016106..f75278b9bf 100644 --- a/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala +++ b/core/src/test/scala/com/salesforce/op/stages/impl/preparators/SanityCheckerTest.scala @@ -448,7 +448,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP .setCorrelationExclusion(CorrelationExclusion.HashedText) .setMinCorrelation(0.0) .setMaxCorrelation(0.8) - .setMaxFeatureCorr(1.0) + .setMaxFeatureCorrelation(1.0) .setMaxCramersV(0.8) .setInput(targetResponse, smartMapVectorized) .getOutput() @@ -481,7 +481,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP .setMinVariance(-0.1) .setMinCorrelation(-0.1) .setMaxCorrelation(1.1) - .setMaxFeatureCorr(0.9) + .setMaxFeatureCorrelation(0.9) .setMaxCramersV(1.0) .setInput(targetLabel, featureVector) .getOutput() @@ -512,7 +512,7 @@ class SanityCheckerTest extends OpEstimatorSpec[OPVector, BinaryModel[RealNN, OP .setMinVariance(-0.1) .setMinCorrelation(0.0) .setMaxCorrelation(0.8) - .setMaxFeatureCorr(1.0) + .setMaxFeatureCorrelation(1.0) .setMaxCramersV(0.8) .setInput(targetResponse, vectorized) .getOutput()