You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/02 22:35:46 UTC

[GitHub] [spark] huaxingao commented on a change in pull request #28710: [SPARK-31893][ML] Add a generic ClassificationSummary trait

huaxingao commented on a change in pull request #28710:
URL: https://github.com/apache/spark/pull/28710#discussion_r434212285



##########
File path: mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala
##########
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.classification
+
+import org.apache.spark.annotation.Since
+import org.apache.spark.ml.functions.checkNonNegativeWeight
+import org.apache.spark.ml.linalg.Vector
+import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, MulticlassMetrics}
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.DoubleType
+
+
+/**
+ * Abstraction for multiclass classification results for a given model.
+ */
+private[classification] trait ClassificationSummary extends Serializable {
+
+  /**
+   * Dataframe output by the model's `transform` method.
+   */
+  @Since("3.1.0")
+  def predictions: DataFrame
+
+  /** Field in "predictions" which gives the prediction of each class. */
+  @Since("3.1.0")
+  def predictionCol: String
+
+  /** Field in "predictions" which gives the true label of each instance (if available). */
+  @Since("3.1.0")
+  def labelCol: String
+
+  /** Field in "predictions" which gives the features of each instance as a vector. */
+  @Since("3.1.0")
+  def featuresCol: String
+
+  /** Field in "predictions" which gives the weight of each instance as a vector. */
+  @Since("3.1.0")
+  def weightCol: String
+
+  @transient private val multiclassMetrics = {
+    if (predictions.schema.fieldNames.contains(weightCol)) {
+      new MulticlassMetrics(
+        predictions.select(
+          col(predictionCol),
+          col(labelCol).cast(DoubleType),
+          checkNonNegativeWeight(col(weightCol).cast(DoubleType))).rdd.map {
+          case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight)
+        })
+    } else {
+      new MulticlassMetrics(
+        predictions.select(
+          col(predictionCol),
+          col(labelCol).cast(DoubleType),
+          lit(1.0)).rdd.map {
+          case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight)
+        })
+    }
+  }
+
+  /**
+   * Returns the sequence of labels in ascending order. This order matches the order used
+   * in metrics which are specified as arrays over labels, e.g., truePositiveRateByLabel.
+   *
+   * Note: In most cases, it will be values {0.0, 1.0, ..., numClasses-1}, However, if the
+   * training set is missing a label, then all of the arrays over labels
+   * (e.g., from truePositiveRateByLabel) will be of length numClasses-1 instead of the
+   * expected numClasses.
+   */
+  @Since("3.1.0")
+  def labels: Array[Double] = multiclassMetrics.labels
+
+  /** Returns true positive rate for each label (category). */
+  @Since("3.1.0")
+  def truePositiveRateByLabel: Array[Double] = recallByLabel
+
+  /** Returns false positive rate for each label (category). */
+  @Since("3.1.0")
+  def falsePositiveRateByLabel: Array[Double] = {
+    multiclassMetrics.labels.map(label => multiclassMetrics.falsePositiveRate(label))
+  }
+
+  /** Returns precision for each label (category). */
+  @Since("3.1.0")
+  def precisionByLabel: Array[Double] = {
+    multiclassMetrics.labels.map(label => multiclassMetrics.precision(label))
+  }
+
+  /** Returns recall for each label (category). */
+  @Since("3.1.0")
+  def recallByLabel: Array[Double] = {
+    multiclassMetrics.labels.map(label => multiclassMetrics.recall(label))
+  }
+
+  /** Returns f-measure for each label (category). */
+  @Since("3.1.0")
+  def fMeasureByLabel(beta: Double): Array[Double] = {
+    multiclassMetrics.labels.map(label => multiclassMetrics.fMeasure(label, beta))
+  }
+
+  /** Returns f1-measure for each label (category). */
+  @Since("3.1.0")
+  def fMeasureByLabel: Array[Double] = fMeasureByLabel(1.0)
+
+  /**
+   * Returns accuracy.
+   * (equals to the total number of correctly classified instances
+   * out of the total number of instances.)
+   */
+  @Since("3.1.0")
+  def accuracy: Double = multiclassMetrics.accuracy
+
+  /**
+   * Returns weighted true positive rate.
+   * (equals to precision, recall and f-measure)
+   */
+  @Since("3.1.0")
+  def weightedTruePositiveRate: Double = weightedRecall
+
+  /** Returns weighted false positive rate. */
+  @Since("3.1.0")
+  def weightedFalsePositiveRate: Double = multiclassMetrics.weightedFalsePositiveRate
+
+  /**
+   * Returns weighted averaged recall.
+   * (equals to precision, recall and f-measure)
+   */
+  @Since("3.1.0")
+  def weightedRecall: Double = multiclassMetrics.weightedRecall
+
+  /** Returns weighted averaged precision. */
+  @Since("3.1.0")
+  def weightedPrecision: Double = multiclassMetrics.weightedPrecision
+
+  /** Returns weighted averaged f-measure. */
+  @Since("3.1.0")
+  def weightedFMeasure(beta: Double): Double = multiclassMetrics.weightedFMeasure(beta)
+
+  /** Returns weighted averaged f1-measure. */
+  @Since("3.1.0")
+  def weightedFMeasure: Double = multiclassMetrics.weightedFMeasure(1.0)
+
+  /**
+   * Convenient method for casting to binary classification summary.
+   * This method will throw an Exception if the summary is not a binary summary.
+   */
+  @Since("3.1.0")
+  def asBinary: BinaryClassificationSummary = this match {
+    case b: BinaryClassificationSummary => b
+    case _ =>
+      throw new RuntimeException("Cannot cast to a binary summary.")
+  }
+}
+
+/**
+ * Abstraction for multiclass classification training results.
+ */
+private[classification] trait ClassificationTrainingSummary extends ClassificationSummary {
+
+  /** objective function (scaled loss + regularization) at each iteration. */
+  @Since("3.1.0")
+  def objectiveHistory: Array[Double]
+
+  /** Number of training iterations. */
+  @Since("3.1.0")
+  def totalIterations: Int = objectiveHistory.length
+}
+
+/**
+ * Abstraction for binary classification results for a given model.
+ */
+trait BinaryClassificationSummary extends ClassificationSummary {
+
+  private val sparkSession = predictions.sparkSession
+  import sparkSession.implicits._
+
+  /**
+   * Field in "predictions" which gives the probability or rawPrediction of each class as a vector.
+   */
+  @Since("3.1.0")
+  def scoreCol: String

Review comment:
       Can't name it to ```probabilityCol``` because not all the classifiers have ```probabilityCol```. If no  ```probability```, will use ```rawPrediction``` instead  to calculate the binary classification metrics. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org