-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-12664][ML] Expose probability in mlp model #17373
Changes from all commits
5beee7b
01987e9
645fdc4
bcb44af
df7439e
eedc647
7704930
5369b08
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -361,17 +361,42 @@ private[ann] trait TopologyModel extends Serializable { | |
* Forward propagation | ||
* | ||
* @param data input data | ||
* @param includeLastLayer Include the last layer in the output. In | ||
* MultilayerPerceptronClassifier, the last layer is always softmax; | ||
* the last layer of outputs is needed for class predictions, but not | ||
* for rawPrediction. | ||
* | ||
* @return array of outputs for each of the layers | ||
*/ | ||
def forward(data: BDM[Double]): Array[BDM[Double]] | ||
def forward(data: BDM[Double], includeLastLayer: Boolean): Array[BDM[Double]] | ||
|
||
/** | ||
* Prediction of the model | ||
* Prediction of the model. See {@link ProbabilisticClassificationModel} | ||
* | ||
* @param data input data | ||
* @param features input features | ||
* @return prediction | ||
*/ | ||
def predict(data: Vector): Vector | ||
def predict(features: Vector): Vector | ||
|
||
/** | ||
* Raw prediction of the model. See {@link ProbabilisticClassificationModel} | ||
* | ||
* @param features input features | ||
* @return raw prediction | ||
* | ||
* Note: This interface is only used for classification Model. | ||
*/ | ||
def predictRaw(features: Vector): Vector | ||
|
||
/** | ||
* Probability of the model. See {@link ProbabilisticClassificationModel} | ||
* | ||
* @param rawPrediction raw prediction vector | ||
* @return probability | ||
* | ||
* Note: This interface is only used for classification Model. | ||
*/ | ||
def raw2ProbabilityInPlace(rawPrediction: Vector): Vector | ||
|
||
/** | ||
* Computes gradient for the network | ||
|
@@ -463,7 +488,7 @@ private[ml] class FeedForwardModel private( | |
private var outputs: Array[BDM[Double]] = null | ||
private var deltas: Array[BDM[Double]] = null | ||
|
||
override def forward(data: BDM[Double]): Array[BDM[Double]] = { | ||
override def forward(data: BDM[Double], includeLastLayer: Boolean): Array[BDM[Double]] = { | ||
// Initialize output arrays for all layers. Special treatment for InPlace | ||
val currentBatchSize = data.cols | ||
// TODO: allocate outputs as one big array and then create BDMs from it | ||
|
@@ -481,7 +506,8 @@ private[ml] class FeedForwardModel private( | |
} | ||
} | ||
layerModels(0).eval(data, outputs(0)) | ||
for (i <- 1 until layerModels.length) { | ||
val end = if (includeLastLayer) layerModels.length else layerModels.length - 1 | ||
for (i <- 1 until end) { | ||
layerModels(i).eval(outputs(i - 1), outputs(i)) | ||
} | ||
outputs | ||
|
@@ -492,7 +518,7 @@ private[ml] class FeedForwardModel private( | |
target: BDM[Double], | ||
cumGradient: Vector, | ||
realBatchSize: Int): Double = { | ||
val outputs = forward(data) | ||
val outputs = forward(data, true) | ||
val currentBatchSize = data.cols | ||
// TODO: allocate deltas as one big array and then create BDMs from it | ||
if (deltas == null || deltas(0).cols != currentBatchSize) { | ||
|
@@ -527,9 +553,20 @@ private[ml] class FeedForwardModel private( | |
|
||
override def predict(data: Vector): Vector = { | ||
val size = data.size | ||
val result = forward(new BDM[Double](size, 1, data.toArray)) | ||
val result = forward(new BDM[Double](size, 1, data.toArray), true) | ||
Vectors.dense(result.last.toArray) | ||
} | ||
|
||
override def predictRaw(data: Vector): Vector = { | ||
val result = forward(new BDM[Double](data.size, 1, data.toArray), false) | ||
Vectors.dense(result(result.length - 2).toArray) | ||
} | ||
|
||
override def raw2ProbabilityInPlace(data: Vector): Vector = { | ||
val dataMatrix = new BDM[Double](data.size, 1, data.toArray) | ||
layerModels.last.eval(dataMatrix, dataMatrix) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This assumes that the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ping: If this proposal sounds good, then can you please update accordingly? |
||
data | ||
} | ||
} | ||
|
||
/** | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,6 +29,7 @@ import org.apache.spark.mllib.linalg.{Vectors => OldVectors} | |
import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} | ||
import org.apache.spark.mllib.util.MLlibTestSparkContext | ||
import org.apache.spark.sql.{Dataset, Row} | ||
import org.apache.spark.sql.functions._ | ||
|
||
class MultilayerPerceptronClassifierSuite | ||
extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { | ||
|
@@ -82,6 +83,47 @@ class MultilayerPerceptronClassifierSuite | |
} | ||
} | ||
|
||
test("Predicted class probabilities: calibration on toy dataset") { | ||
val layers = Array[Int](4, 5, 2) | ||
|
||
val strongDataset = Seq( | ||
(Vectors.dense(1, 2, 3, 4), 0d, Vectors.dense(1d, 0d)), | ||
(Vectors.dense(4, 3, 2, 1), 1d, Vectors.dense(0d, 1d)), | ||
(Vectors.dense(1, 1, 1, 1), 0d, Vectors.dense(.5, .5)), | ||
(Vectors.dense(1, 1, 1, 1), 1d, Vectors.dense(.5, .5)) | ||
).toDF("features", "label", "expectedProbability") | ||
val trainer = new MultilayerPerceptronClassifier() | ||
.setLayers(layers) | ||
.setBlockSize(1) | ||
.setSeed(123L) | ||
.setMaxIter(100) | ||
.setSolver("l-bfgs") | ||
val model = trainer.fit(strongDataset) | ||
val result = model.transform(strongDataset) | ||
result.select("probability", "expectedProbability").collect().foreach { | ||
case Row(p: Vector, e: Vector) => | ||
assert(p ~== e absTol 1e-3) | ||
} | ||
} | ||
|
||
test("test model probability") { | ||
val layers = Array[Int](2, 5, 2) | ||
val trainer = new MultilayerPerceptronClassifier() | ||
.setLayers(layers) | ||
.setBlockSize(1) | ||
.setSeed(123L) | ||
.setMaxIter(100) | ||
.setSolver("l-bfgs") | ||
val model = trainer.fit(dataset) | ||
model.setProbabilityCol("probability") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's the default already, right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ping --- this should not be necessary |
||
val result = model.transform(dataset) | ||
val features2prob = udf { features: Vector => model.mlpModel.predict(features) } | ||
result.select(features2prob(col("features")), col("probability")).collect().foreach { | ||
case Row(p1: Vector, p2: Vector) => | ||
assert(p1 ~== p2 absTol 1e-3) | ||
} | ||
} | ||
|
||
test("Test setWeights by training restart") { | ||
val dataFrame = Seq( | ||
(Vectors.dense(0.0, 0.0), 0.0), | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
add
raw2ProbabilityInPlace
, what it compute is:directly call the last layer function to compute it.