Skip to content

Commit

Permalink
Added value to OpVectorColumnMetadata for numeric column descriptors (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
leahmcguire authored and tovbinm committed Aug 27, 2018
1 parent a4afce4 commit f37113b
Show file tree
Hide file tree
Showing 31 changed files with 129 additions and 194 deletions.
4 changes: 2 additions & 2 deletions core/src/main/scala/com/salesforce/op/ModelInsights.scala
Original file line number Diff line number Diff line change
Expand Up @@ -527,7 +527,7 @@ case object ModelInsights {
Insights(
derivedFeatureName = h.columnName,
stagesApplied = h.parentFeatureStages,
derivedFeatureGroup = h.indicatorGroup,
derivedFeatureGroup = h.grouping,
derivedFeatureValue = h.indicatorValue,
excluded = Option(s.dropped.contains(h.columnName)),
corr = getCorr(s.correlationsWLabel, h.columnName),
Expand Down Expand Up @@ -555,7 +555,7 @@ case object ModelInsights {
Insights(
derivedFeatureName = h.columnName,
stagesApplied = h.parentFeatureStages,
derivedFeatureGroup = h.indicatorGroup,
derivedFeatureGroup = h.grouping,
derivedFeatureValue = h.indicatorValue,
contribution = contributions.map(_.applyOrElse(h.index, Seq.empty)) // nothing dropped without sanity check
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -281,8 +281,9 @@ class DateListVectorizer[T <: OPList[Long]]
} yield OpVectorColumnMetadata(
parentFeatureName = col.parentFeatureName,
parentFeatureType = col.parentFeatureType,
indicatorGroup = col.parentFeatureName,
indicatorValue = Option(pivotValue)
grouping = col.parentFeatureName,
indicatorValue = Option(pivotValue),
descriptorValue = None
)
vectorMeta.withColumns(updatedCols)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ class DecisionTreeNumericMapBucketizer[N, I2 <: OPMap[N]]
val cols: Array[Array[OpVectorColumnMetadata]] = allSplits.map { case (key, split) =>
makeVectorColumnMetadata(
input = in2,
indicatorGroup = Some(key),
grouping = Some(key),
bucketLabels = split.bucketLabels,
trackNulls = $(trackNulls),
trackInvalid = split.shouldSplit && $(trackInvalid)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ class GeolocationMapVectorizer
} yield new OpVectorColumnMetadata(
parentFeatureName = col.parentFeatureName,
parentFeatureType = col.parentFeatureType,
indicatorGroup = Option(key),
grouping = Option(key),
indicatorValue = None
)
meta.withColumns(cols.toArray)
Expand All @@ -79,7 +79,7 @@ class GeolocationMapVectorizer
col :+ OpVectorColumnMetadata(
parentFeatureName = head.parentFeatureName,
parentFeatureType = head.parentFeatureType,
indicatorGroup = head.indicatorGroup,
grouping = head.grouping,
indicatorValue = Some(TransmogrifierDefaults.NullString)
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ private[op] trait NumericBucketizerMetadata {
val cols = makeVectorColumnMetadata(
input = input,
bucketLabels = bucketLabels,
indicatorGroup = Some(input.name),
grouping = Some(input.name),
trackInvalid = trackInvalid,
trackNulls = trackNulls
)
Expand All @@ -157,11 +157,11 @@ private[op] trait NumericBucketizerMetadata {
protected def makeVectorColumnMetadata(
input: TransientFeature,
bucketLabels: Array[String],
indicatorGroup: Option[String],
grouping: Option[String],
trackInvalid: Boolean,
trackNulls: Boolean
): Array[OpVectorColumnMetadata] = {
val meta = input.toColumnMetaData(true).copy(indicatorGroup = indicatorGroup)
val meta = input.toColumnMetaData(true).copy(grouping = grouping)
val bucketLabelCols = bucketLabels.map(bucketLabel => meta.copy(indicatorValue = Option(bucketLabel)))
val trkInvCol = if (trackInvalid) Seq(meta.copy(indicatorValue = Some(TransmogrifierDefaults.OtherString))) else Nil
val trackNullCol = if (trackNulls) Seq(meta) else Nil
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ private[op] trait HashingFun {
OpVectorColumnMetadata(
parentFeatureName = features.map(_.name),
parentFeatureType = features.map(_.typeName),
indicatorGroup = None,
grouping = None,
indicatorValue = None
)
}.toArray
Expand Down Expand Up @@ -324,7 +324,7 @@ private[op] trait MapHashingFun extends HashingFun {
OpVectorColumnMetadata(
parentFeatureName = features.map(_.name),
parentFeatureType = features.map(_.typeName),
indicatorGroup = None,
grouping = None,
indicatorValue = None
)
}.toArray
Expand All @@ -333,13 +333,13 @@ private[op] trait MapHashingFun extends HashingFun {
(keys, f) <- allKeys.toArray.zip(features)
key <- keys
i <- 0 until numHashes
} yield f.toColumnMetaData().copy(indicatorGroup = Option(key))
} yield f.toColumnMetaData().copy(grouping = Option(key))
}
val nullColumns = if (shouldTrackNulls) {
for {
(keys, f) <- allKeys.toArray.zip(features)
key <- keys
} yield f.toColumnMetaData(isNull = true).copy(indicatorGroup = Option(key))
} yield f.toColumnMetaData(isNull = true).copy(grouping = Option(key))
} else Array.empty[OpVectorColumnMetadata]

hashColumns ++ nullColumns
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -290,7 +290,7 @@ trait MapVectorizerFuns[A, T <: OPMap[A]] extends VectorizerDefaults with MapPiv
} yield new OpVectorColumnMetadata(
parentFeatureName = col.parentFeatureName,
parentFeatureType = col.parentFeatureType,
indicatorGroup = Option(key),
grouping = Option(key),
indicatorValue = None
)
meta.withColumns(cols.toArray)
Expand All @@ -305,7 +305,7 @@ trait MapVectorizerFuns[A, T <: OPMap[A]] extends VectorizerDefaults with MapPiv
OpVectorColumnMetadata(
parentFeatureName = col.parentFeatureName,
parentFeatureType = col.parentFeatureType,
indicatorGroup = col.indicatorGroup,
grouping = col.grouping,
indicatorValue = Some(TransmogrifierDefaults.NullString)
)
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ class OpCountVectorizer(uid: String = UID[OpCountVectorizer])
} yield OpVectorColumnMetadata(
parentFeatureName = Seq(f.name),
parentFeatureType = Seq(f.typeName),
indicatorGroup = None, // TODO do we want to test each word for label pred?
grouping = None, // TODO do we want to test each word for label pred?
indicatorValue = Option(word)
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ class TextMapNullEstimator[T <: OPMap[String]]
} yield OpVectorColumnMetadata(
parentFeatureName = Seq(tf.name),
parentFeatureType = Seq(tf.typeName),
indicatorGroup = Option(key),
grouping = Option(key),
indicatorValue = Option(OpVectorColumnMetadata.NullString)
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -644,7 +644,7 @@ trait MapStringPivotHelper extends SaveOthersParams {
} yield OpVectorColumnMetadata(
parentFeatureName = Seq(f.name),
parentFeatureType = Seq(f.typeName),
indicatorGroup = Option(key),
grouping = Option(key),
indicatorValue = Option(value)
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -278,8 +278,8 @@ class SanityChecker(uid: String = UID[SanityChecker])
val nullGroups = for {
col <- metaCols
if col.isNullIndicator
indicatorGroup <- col.indicatorGroup
} yield (indicatorGroup, (col, col.index))
grouping <- col.grouping
} yield (grouping, (col, col.index))

nullGroups.groupBy(_._1).foreach {
case (group, cols) =>
Expand Down Expand Up @@ -387,7 +387,7 @@ class SanityChecker(uid: String = UID[SanityChecker])

// Calculate groups to remove separately. This is for more complicated checks where you can't determine whether
// to remove a feature from a single column stats (eg. associate rule confidence/support check)
val groupByGroups = stats.groupBy(_.column.flatMap(_.indicatorGroup))
val groupByGroups = stats.groupBy(_.column.flatMap(_.grouping))
val ruleConfGroupsToDrop = groupByGroups.toSeq.flatMap{
case (Some(group), colStats) =>
val colsToRemove = colStats.filter(f =>
Expand Down Expand Up @@ -463,19 +463,19 @@ class SanityChecker(uid: String = UID[SanityChecker])
logInfo("Label is assumed to be categorical since either categoricalLabel = true or " +
"number of distinct labels < count * 0.1")

// Only perform Cramer's V calculation on columns that have an indicatorGroup and indicatorValue defined (right
// now, the only things that will have indicatorValue defined and indicatorGroup be None is numeric maps)
val columnsWithIndicator = columnMeta.filter(f => f.indicatorGroup.isDefined && f.indicatorValue.isDefined)
val colIndicesByIndicatorGroup =
// Only perform Cramer's V calculation on columns that have an grouping and indicatorValue defined (right
// now, the only things that will have indicatorValue defined and grouping be None is numeric maps)
val columnsWithIndicator = columnMeta.filter(f => f.grouping.isDefined && f.indicatorValue.isDefined)
val colIndicesByGrouping =
columnsWithIndicator
.map { meta => meta.indicatorGroup.get -> meta }
.map { meta => meta.grouping.get -> meta }
.groupBy(_._1)
// Keep track of the group, column name, column index, and whether the parent was a MultiPickList or not
.map { case (group, cols) => (group, cols.map(_._2.makeColName()), cols.map(_._2.index),
cols.exists(_._2.hasParentOfSubType[MultiPickList]))
}

colIndicesByIndicatorGroup.map {
colIndicesByGrouping.map {
case (group, colNames, valueIndices, isMpl) =>
val groupContingency =
if (valueIndices.length == 1) {
Expand Down Expand Up @@ -601,7 +601,7 @@ class SanityChecker(uid: String = UID[SanityChecker])
// Indices are determined to be hashed if they come from Text/TextArea types (or their maps) and don't have
// an indicator group or indicator value (indicating that they are not pivoted out by the SmartTextVectorizer
// TODO: Find a better way to do this with the feature history
f.indicatorGroup.isEmpty && f.indicatorValue.isEmpty &&
f.grouping.isEmpty && f.indicatorValue.isEmpty &&
f.parentFeatureType.exists { t =>
val tt = FeatureType.featureTypeTag(t)
tt.tpe =:= typeTag[Text].tpe || tt.tpe =:= typeTag[TextArea].tpe ||
Expand Down Expand Up @@ -814,7 +814,7 @@ private[op] case class ColumnStatistics
s"Max association rule confidence $conf is above threshold of $maxRuleConfidence and support $sup is " +
s"above the required support threshold of $minRequiredRuleSupport"
},
column.flatMap(_.indicatorGroup).filter(removedGroups.contains(_)).map(ig =>
column.flatMap(_.grouping).filter(removedGroups.contains(_)).map(ig =>
s"other feature in indicator group $ig flagged for removal via rule confidence checks"
)
).flatten
Expand Down Expand Up @@ -842,7 +842,7 @@ private[op] case class ColumnStatistics
def isTextSharedHash(metadata: OpVectorColumnMetadata): Boolean = {
val isDerivedFromText = metadata.hasParentOfType[Text] || metadata.hasParentOfType[TextArea] ||
metadata.hasParentOfType[TextMap] || metadata.hasParentOfType[TextAreaMap]
isDerivedFromText && metadata.indicatorGroup.isEmpty && metadata.indicatorValue.isEmpty
isDerivedFromText && metadata.grouping.isEmpty && metadata.indicatorValue.isEmpty
}

override def toString: String = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -319,23 +319,6 @@ case object SanityCheckerSummary {
)
}

@deprecated("CategoricalStats replaced by Array[CategoricalGroupStats]", "3.3.0")
private def categoricalStatsFromMetadata(meta: Metadata): CategoricalStats = {
val wrapped = meta.wrapped
CategoricalStats(
categoricalFeatures = wrapped.getArray[String](SanityCheckerNames.CategoricalFeatures),
cramersVs = wrapped.getArray[Double](SanityCheckerNames.CramersV),
mutualInfos = wrapped.getArray[Double](SanityCheckerNames.MutualInfo),
pointwiseMutualInfos = meta.getMetadata(SanityCheckerNames.PointwiseMutualInfoAgainstLabel)
.underlyingMap.asInstanceOf[LabelWiseValues.Type],
counts = if (meta.contains(SanityCheckerNames.CountMatrix)) {
meta.getMetadata(SanityCheckerNames.CountMatrix)
.underlyingMap.asInstanceOf[LabelWiseValues.Type]
}
else OpStatistics.LabelWiseValues.empty
)
}

private def categoricalGroupStatsFromMetadata(meta: Metadata): CategoricalGroupStats = {
val wrapped = meta.wrapped
CategoricalGroupStats(
Expand Down Expand Up @@ -373,41 +356,7 @@ case object SanityCheckerSummary {
} match {
case Success(summary) => summary
// Parse it under the old format
case Failure(_) =>
val oldCatStats = categoricalStatsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.CategoricalStats))
SanityCheckerSummary(
// Try to parse correlations under an even older OP 3.1 format (for PLS backwards compatibility)
correlationsWLabel = Try(
correlationsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.CorrelationsWLabel))
) match {
case Success(corr) => corr
case Failure(_) =>
Correlations(
featuresIn = wrapped.get[Metadata](SanityCheckerNames.CorrelationsWLabel).wrapped
.getArray[String](SanityCheckerNames.FeaturesIn).toSeq,
values = wrapped.get[Metadata](SanityCheckerNames.CorrelationsWLabel).wrapped
.getArray[Double](SanityCheckerNames.Values).toSeq,
nanCorrs = wrapped.getArray[String](SanityCheckerNames.CorrelationsWLabelIsNaN).toSeq,
corrType = wrapped.get[String](SanityCheckerNames.CorrelationType) match {
case CorrelationType.Pearson.`sparkName` => CorrelationType.Pearson
case CorrelationType.Spearman.`sparkName` => CorrelationType.Spearman
}
)
},
dropped = wrapped.getArray[String](SanityCheckerNames.Dropped).toSeq,
featuresStatistics = statisticsFromMetadata(wrapped.get[Metadata](SanityCheckerNames.FeaturesStatistics)),
names = wrapped.getArray[String](SanityCheckerNames.Names).toSeq,
categoricalStats = Array(CategoricalGroupStats(
group = "Unknown - deprecated metadata",
categoricalFeatures = oldCatStats.categoricalFeatures,
contingencyMatrix = oldCatStats.counts,
pointwiseMutualInfo = oldCatStats.pointwiseMutualInfos,
cramersV = oldCatStats.cramersVs.head,
mutualInfo = oldCatStats.mutualInfos.head,
maxRuleConfidences = Array.empty[Double],
supports = Array.empty[Double]
))
)
case Failure(_) => throw new IllegalArgumentException(s"failed to parse SanityCheckerSummary from $meta")
}
}
}
28 changes: 0 additions & 28 deletions core/src/test/resources/SanityCheckerSummaryMetadata_pre3.3.0.json

This file was deleted.

4 changes: 2 additions & 2 deletions core/src/test/scala/com/salesforce/op/ModelInsightsTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -382,13 +382,13 @@ class ModelInsightsTest extends FlatSpec with PassengerSparkFixtureTest {
OpVectorColumnMetadata(
parentFeatureName = Seq("f1"),
parentFeatureType = Seq(classOf[Real].getName),
indicatorGroup = None,
grouping = None,
indicatorValue = None
) +: Array("f2", "f3").map { name =>
OpVectorColumnMetadata(
parentFeatureName = Seq("f0"),
parentFeatureType = Seq(classOf[PickList].getName),
indicatorGroup = Option("f0"),
grouping = Option("f0"),
indicatorValue = Option(name)
)
},
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/com/salesforce/op/OpWorkflowTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -537,7 +537,7 @@ object NormEstimatorTest {
val columnMeta = OpVectorColumnMetadata(
parentFeatureName = Seq("parentFeature"),
parentFeatureType = Seq(FeatureTypeDefaults.Real.getClass.getName),
indicatorGroup = Some("indicator_group"),
grouping = Some("indicator_group"),
indicatorValue = None
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ class DateMapVectorizerTest extends FlatSpec with TestSparkContext {
}
val meta = OpVectorMetadata(vector.name, transformed.schema(vector.name).metadata)
meta.columns.length shouldBe 3
meta.columns.map(_.indicatorGroup) should contain theSameElementsAs Array(Option("a"), Option("b"), Option("c"))
meta.columns.map(_.grouping) should contain theSameElementsAs Array(Option("a"), Option("b"), Option("c"))

val vector2 = f1.vectorize(defaultValue = 0, referenceDate = moment, trackNulls = true)
val transformed2 = new OpWorkflow().setResultFeatures(vector2).transform(ds)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ class DecisionTreeNumericMapBucketizerTest extends OpEstimatorSpec[OPVector,
featureVectorMeta.columns.foreach{ col =>
col.parentFeatureName should contain theSameElementsAs Seq("currencyMap")
col.parentFeatureType should contain theSameElementsAs Seq("com.salesforce.op.features.types.CurrencyMap")
col.indicatorGroup shouldBe Some("c0")
col.grouping shouldBe Some("c0")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -390,15 +390,15 @@ object OPMapVectorizerTestHelper extends Matchers {

// val baseIndicesToCompare: Array[Int] = baseColMetaArray.filterNot(_.isNullIndicator).map(_.index).sorted
val baseIndicesToCompare: Array[Int] = baseColMetaArray
.map(f => (f.parentFeatureName.head, f.indicatorValue, f.indicatorGroup) match {
.map(f => (f.parentFeatureName.head, f.indicatorValue, f.grouping) match {
case (pfName, Some(iv), Some(ig)) => (pfName + ig + iv, f.index)
case (pfName, Some(iv), None) => (pfName + iv, f.index)
case (pfName, None, Some(ig)) => (pfName + ig, f.index)
case (pfName, None, None) => (pfName, f.index)
}).sortBy(_._1).map(_._2)
// Also need to sort map vectorized indices by feature name since they can come out in arbitrary orders
val mapIndicesToCompare: Array[Int] = mapColMetaArray
.map(f => (f.parentFeatureName.head, f.indicatorValue, f.indicatorGroup) match {
.map(f => (f.parentFeatureName.head, f.indicatorValue, f.grouping) match {
case (pfName, Some(iv), Some(ig)) => (pfName + ig + iv, f.index)
case (pfName, Some(iv), None) => (pfName + iv, f.index)
case (pfName, None, Some(ig)) => (pfName + ig, f.index)
Expand Down
Loading

0 comments on commit f37113b

Please sign in to comment.