From bfade12f198a159dd79f5d8eb690615526376ae7 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 23 Nov 2014 23:18:52 -0800 Subject: [PATCH 01/30] Added lots of classes for new ML API: Abstract classes for learning algorithms: * Classifier * Regressor * Predictor Traits for learning algorithms * HasDefaultEstimator * IterativeEstimator * IterativeSolver * ProbabilisticClassificationModel * WeakLearner Concrete classes: learning algorithms * AdaBoost (partly implemented) * NaiveBayes (rough implementation) * LinearRegression * LogisticRegression (updated to use new abstract classes) Concrete classes: evaluation * ClassificationEvaluator * RegressionEvaluator * PredictionEvaluator Concrete classes: other * LabeledPoint (adding weight to the old LabeledPoint) --- .../org/apache/spark/ml/LabeledPoint.scala | 24 ++ .../spark/ml/classification/AdaBoost.scala | 208 ++++++++++++++++++ .../spark/ml/classification/Classifier.scala | 73 ++++++ .../classification/LogisticRegression.scala | 120 +++++----- .../spark/ml/classification/NaiveBayes.scala | 67 ++++++ .../BinaryClassificationEvaluator.scala | 12 +- .../evaluation/ClassificationEvaluator.scala | 57 +++++ .../ml/evaluation/RegressionEvaluator.scala | 57 +++++ .../evaluation/impl/PredictionEvaluator.scala | 59 +++++ .../impl/estimator/HasDefaultEvaluator.scala | 12 + .../impl/estimator/IterativeEstimator.scala | 18 ++ .../ml/impl/estimator/IterativeSolver.scala | 54 +++++ .../spark/ml/impl/estimator/Predictor.scala | 135 ++++++++++++ .../ProbabilisticClassificationModel.scala | 12 + .../spark/ml/impl/estimator/WeakLearner.scala | 19 ++ .../apache/spark/ml/param/sharedParams.scala | 13 ++ .../ml/regression/LinearRegression.scala | 72 ++++++ .../spark/ml/regression/Regressor.scala | 33 +++ .../spark/ml/tuning/CrossValidator.scala | 14 +- 19 files changed, 1001 insertions(+), 58 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala new file mode 100644 index 0000000000000..47a6c71b78916 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala @@ -0,0 +1,24 @@ +package org.apache.spark.ml + +import org.apache.spark.mllib.linalg.Vector + +/** + * Class that represents an instance (data point) for prediction tasks. + * + * @param label Label to predict + * @param features List of features describing this instance + * @param weight Instance weight + */ +case class LabeledPoint(label: Double, features: Vector, weight: Double) { + + /** Default constructor which sets instance weight to 1.0 */ + def this(label: Double, features: Vector) = this(label, features, 1.0) + + override def toString: String = { + "(%s,%s,%s)".format(label, features, weight) + } +} + +object LabeledPoint { + def apply(label: Double, features: Vector) = new LabeledPoint(label, features) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala new file mode 100644 index 0000000000000..5517cd95cf42f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala @@ -0,0 +1,208 @@ +package org.apache.spark.ml.classification + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.ml.evaluation.ClassificationEvaluator +import org.apache.spark.ml.param.{HasWeightCol, Param, ParamMap, HasMaxIter} +import org.apache.spark.ml.impl.estimator.{ProbabilisticClassificationModel, WeakLearner, + IterativeEstimator, IterativeSolver} + + +private[classification] trait AdaBoostParams extends ClassifierParams + with HasMaxIter with HasWeightCol { + + /** param for weak learner type */ + val weakLearner: Param[Classifier[_, _]] = + new Param(this, "weakLearner", "weak learning algorithm") + def getWeakLearner: Classifier[_, _] = get(weakLearner) + + /** param for weak learner param maps */ + val weakLearnerParamMap: Param[ParamMap] = + new Param(this, "weakLearnerParamMap", "param map for the weak learner") + def getWeakLearnerParamMap: ParamMap = get(weakLearnerParamMap) + + override def validate(paramMap: ParamMap): Unit = { + // TODO: Check maxIter, weakLearner, weakLearnerParamMap, weightCol + // Check: If the weak learner does not extend WeakLearner, then featuresColName should be + // castable to FeaturesType. + } +} + + +/** + * AdaBoost + * + * Developer notes: + * - If the weak learner implements the [[WeakLearner]] + */ +class AdaBoost extends Classifier[AdaBoost, AdaBoostModel] + with AdaBoostParams + with IterativeEstimator[AdaBoostModel] { + + def setMaxIter(value: Int): this.type = set(maxIter, value) + def setWeightCol(value: String): this.type = set(weightCol, value) + def setWeakLearner(value: Classifier[_, _]): this.type = set(weakLearner, value) + def setWeakLearnerParamMap(value: ParamMap): this.type = set(weakLearnerParamMap, value) + + /** + * Extract LabeledPoints, using the weak learner's native feature representation if possible. + * @param paramMap Complete paramMap (after combining with the internal paramMap) + */ + private def extractLabeledPoints(dataset: SchemaRDD, paramMap: ParamMap): RDD[LabeledPoint] = { + import dataset.sqlContext._ + val featuresColName = paramMap(featuresCol) + val wl = paramMap(weakLearner) + val featuresRDD: RDD[Vector] = wl match { + case wlTagged: WeakLearner => + val wlParamMap = paramMap(weakLearnerParamMap) + val wlFeaturesColName = wlParamMap(wl.featuresCol) + val origFeaturesRDD = dataset.select(featuresColName.attr).as(wlFeaturesColName.attr) + wlTagged.getNativeFeatureRDD(origFeaturesRDD, wlParamMap) + case _ => + dataset.select(featuresColName.attr).map { case Row(features: Vector) => features } + } + + val labelColName = paramMap(labelCol) + if (paramMap.contains(weightCol)) { + val weightColName = paramMap(weightCol) + dataset.select(labelColName.attr, weightColName.attr) + .zip(featuresRDD).map { case (Row(label: Double, weight: Double), features: Vector) => + LabeledPoint(label, features, weight) + } + } else { + dataset.select(labelColName.attr) + .zip(featuresRDD).map { case (Row(label: Double), features: Vector) => + LabeledPoint(label, features) + } + } + } + + // From Classifier + override def fit(dataset: SchemaRDD, paramMap: ParamMap): AdaBoostModel = { + val map = this.paramMap ++ paramMap + val labeledPoints: RDD[LabeledPoint] = extractLabeledPoints(dataset, map) + train(labeledPoints, paramMap) + } + + // From IterativeEstimator + override private[ml] def createSolver(dataset: SchemaRDD, paramMap: ParamMap): AdaBoostSolver = { + val map = this.paramMap ++ paramMap + val labeledPoints: RDD[LabeledPoint] = extractLabeledPoints(dataset, map) + new AdaBoostSolver(labeledPoints, this, map) + } + + // From Predictor + override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): AdaBoostModel = { + val map = this.paramMap ++ paramMap + val solver = new AdaBoostSolver(dataset, this, map) + while (solver.step()) { } + solver.currentModel + } +} + + +class AdaBoostModel private[ml] ( + val weakHypotheses: Array[ClassificationModel[_]], + val weakHypothesisWeights: Array[Double], + override val parent: AdaBoost, + override val fittingParamMap: ParamMap) + extends ClassificationModel[AdaBoostModel] + with ProbabilisticClassificationModel + with AdaBoostParams { + + require(weakHypotheses.size != 0) + require(weakHypotheses.size == weakHypothesisWeights.size) + + // From Classifier.Model: + override val numClasses: Int = weakHypotheses(0).numClasses + + require(weakHypotheses.forall(_.numClasses == numClasses)) + + private val margin: Vector => Double = (features) => { + weakHypotheses.zip(weakHypothesisWeights) + .foldLeft(0.0) { case (total: Double, (wh: ClassificationModel[_], weight: Double)) => + val pred = if (wh.predict(features) == 1.0) 1.0 else -1.0 + total + weight * pred + } + } + + private val score: Vector => Double = (features) => { + val m = margin(features) + 1.0 / (1.0 + math.exp(-2.0 * m)) + } + + override def predictProbabilities(features: Vector): Vector = { + val s = score(features) + Vectors.dense(Array(1.0 - s, s)) + } + + override def predictRaw(features: Vector): Vector = { + val m = margin(features) + Vectors.dense(Array(-m, m)) + } +} + + +private[ml] class AdaBoostSolver( + val origData: RDD[LabeledPoint], + val parent: AdaBoost, + val paramMap: ParamMap) extends IterativeSolver[AdaBoostModel] { + + private val weakHypotheses = new ArrayBuffer[ClassificationModel[_]] + private val weakHypothesisWeights = new ArrayBuffer[Double] + + private val wl: Classifier[_, _] = paramMap(parent.weakLearner) + private val wlParamMap = paramMap(parent.weakLearnerParamMap) + override val maxIterations: Int = paramMap(parent.maxIter) + + // TODO: Decide if this alg should cache data, or if that should be left to the user. + + // TODO: check for weights = 0 + // TODO: EDITING HERE NOW: switch to log weights + private var logInstanceWeights: RDD[Double] = origData.map(lp => math.log(lp.weight)) + + override def stepImpl(): Boolean = ??? /*{ + // Check if the weak learner takes instance weights. + val wlDataset = wl match { + case wlWeighted: HasWeightCol => + origData.zip(logInstanceWeights).map { case (lp: LabeledPoint, logWeight: Double) => + LabeledPoint(lp.label, lp.features, weight) + } + case _ => + // Subsample data to simulate the current instance weight distribution. + // TODO: This needs to be done before AdaBoost is committed. + throw new NotImplementedError( + "AdaBoost currently requires that the weak learning algorithm accept instance weights.") + } + // Train the weak learning algorithm. + val weakHypothesis: ClassificationModel[_] = wl match { + case wlTagged: WeakLearner[_] => + // This lets the weak learner know that the features are in its native format. + wlTagged.trainNative(wlDataset, wlParamMap).asInstanceOf[ClassificationModel[_]] + case _ => + wl.train(wlDataset, wlParamMap).asInstanceOf[ClassificationModel[_]] + } + // Add the weighted weak hypothesis to the ensemble. + // TODO: Handle instance weights. + val predictionsAndLabels = wlDataset.map(lp => weakHypothesis.predict(lp.features)) + .zip(wlDataset.map(_.label)) + val eps = ClassificationEvaluator.computeMetric(predictionsAndLabels, "accuracy") + val alpha = 0.5 * (math.log(1.0 - eps) - math.log(eps)) // TODO: handle eps near 0 + weakHypotheses += weakHypothesis + weakHypothesisWeights += alpha + // Update weights. + val newInstanceWeights = instanceWeights.zip(predictionsAndLabels).map { + case (weight: Double, (pred: Double, label: Double)) => + ??? + } + + }*/ + + override def currentModel: AdaBoostModel = { + new AdaBoostModel(weakHypotheses.toArray, weakHypothesisWeights.toArray, parent, paramMap) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala new file mode 100644 index 0000000000000..5d146f6724958 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -0,0 +1,73 @@ +/* + * 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.AlphaComponent +import org.apache.spark.ml.evaluation.ClassificationEvaluator +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml._ +import org.apache.spark.ml.impl.estimator.{HasDefaultEvaluator, PredictionModel, Predictor, + PredictorParams} +import org.apache.spark.rdd.RDD + +@AlphaComponent +private[classification] trait ClassifierParams extends PredictorParams + +/** + * Single-label binary or multiclass classification + */ +abstract class Classifier[Learner <: Classifier[Learner, M], M <: ClassificationModel[M]] + extends Predictor[Learner, M] + with ClassifierParams + with HasDefaultEvaluator { + + override def defaultEvaluator: Evaluator = new ClassificationEvaluator +} + + +private[ml] abstract class ClassificationModel[M <: ClassificationModel[M]] + extends PredictionModel[M] with ClassifierParams { + + def numClasses: Int + + /** + * Predict label for the given features. Labels are indexed {0, 1, ..., numClasses - 1}. + * This default implementation for classification predicts the index of the maximum value + * from [[predictRaw()]]. + */ + override def predict(features: Vector): Double = { + predictRaw(features).toArray.zipWithIndex.maxBy(_._1)._2 + } + + /** + * Raw prediction for each possible label + * @return vector where element i is the raw score for label i + */ + def predictRaw(features: Vector): Vector + + /** + * Compute this model's accuracy on the given dataset. + */ + def accuracy(dataset: RDD[LabeledPoint]): Double = { + // TODO: Handle instance weights. + val predictionsAndLabels = dataset.map(lp => predict(lp.features)) + .zip(dataset.map(_.label)) + ClassificationEvaluator.computeMetric(predictionsAndLabels, "accuracy") + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index b46a5cd8bdf29..19b384cf04316 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -18,11 +18,12 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml._ +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.ml.impl.estimator.ProbabilisticClassificationModel import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors, BLAS, Vector} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -33,46 +34,31 @@ import org.apache.spark.storage.StorageLevel * Params for logistic regression. */ @AlphaComponent -private[classification] trait LogisticRegressionParams extends Params - with HasRegParam with HasMaxIter with HasLabelCol with HasThreshold with HasFeaturesCol - with HasScoreCol with HasPredictionCol { +private[classification] trait LogisticRegressionParams extends ClassifierParams + with HasRegParam with HasMaxIter with HasThreshold with HasScoreCol { - /** - * Validates and transforms the input schema with the provided param map. - * @param schema input schema - * @param paramMap additional parameters - * @param fitting whether this is in fitting - * @return output schema - */ - protected def validateAndTransformSchema( + override protected def validateAndTransformSchema( schema: StructType, paramMap: ParamMap, fitting: Boolean): StructType = { + val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting) val map = this.paramMap ++ paramMap - val featuresType = schema(map(featuresCol)).dataType - // TODO: Support casting Array[Double] and Array[Float] to Vector. - require(featuresType.isInstanceOf[VectorUDT], - s"Features column ${map(featuresCol)} must be a vector column but got $featuresType.") - if (fitting) { - val labelType = schema(map(labelCol)).dataType - require(labelType == DoubleType, - s"Cannot convert label column ${map(labelCol)} of type $labelType to a double column.") - } - val fieldNames = schema.fieldNames + val fieldNames = parentSchema.fieldNames require(!fieldNames.contains(map(scoreCol)), s"Score column ${map(scoreCol)} already exists.") - require(!fieldNames.contains(map(predictionCol)), - s"Prediction column ${map(predictionCol)} already exists.") - val outputFields = schema.fields ++ Seq( - StructField(map(scoreCol), DoubleType, false), - StructField(map(predictionCol), DoubleType, false)) + val outputFields = parentSchema.fields ++ Seq( + StructField(map(scoreCol), DoubleType, nullable = false)) StructType(outputFields) } } + /** * Logistic regression. */ -class LogisticRegression extends Estimator[LogisticRegressionModel] with LogisticRegressionParams { +class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressionModel] + with LogisticRegressionParams { + + // TODO: Extend IterativeEstimator setRegParam(0.1) setMaxIter(100) @@ -80,35 +66,31 @@ class LogisticRegression extends Estimator[LogisticRegressionModel] with Logisti def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) - def setLabelCol(value: String): this.type = set(labelCol, value) def setThreshold(value: Double): this.type = set(threshold, value) - def setFeaturesCol(value: String): this.type = set(featuresCol, value) def setScoreCol(value: String): this.type = set(scoreCol, value) - def setPredictionCol(value: String): this.type = set(predictionCol, value) - override def fit(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val instances = dataset.select(map(labelCol), map(featuresCol)) - .map { case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - }.persist(StorageLevel.MEMORY_AND_DISK) + def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LogisticRegressionModel = { + val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => + org.apache.spark.mllib.regression.LabeledPoint(label, features) + } + val handlePersistence = oldDataset.getStorageLevel == StorageLevel.NONE + if (handlePersistence) { + oldDataset.persist(StorageLevel.MEMORY_AND_DISK) + } val lr = new LogisticRegressionWithLBFGS lr.optimizer - .setRegParam(map(regParam)) - .setNumIterations(map(maxIter)) - val lrm = new LogisticRegressionModel(this, map, lr.run(instances).weights) - instances.unpersist() - // copy model params - Params.inheritValues(map, this, lrm) + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) + val model = lr.run(oldDataset) + val lrm = new LogisticRegressionModel(this, paramMap, model.weights, model.intercept) + if (handlePersistence) { + oldDataset.unpersist() + } lrm } - - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = true) - } } + /** * :: AlphaComponent :: * Model produced by [[LogisticRegression]]. @@ -117,16 +99,22 @@ class LogisticRegression extends Estimator[LogisticRegressionModel] with Logisti class LogisticRegressionModel private[ml] ( override val parent: LogisticRegression, override val fittingParamMap: ParamMap, - weights: Vector) - extends Model[LogisticRegressionModel] with LogisticRegressionParams { + val weights: Vector, + val intercept: Double) + extends ClassificationModel[LogisticRegressionModel] + with ProbabilisticClassificationModel + with LogisticRegressionParams { def setThreshold(value: Double): this.type = set(threshold, value) - def setFeaturesCol(value: String): this.type = set(featuresCol, value) def setScoreCol(value: String): this.type = set(scoreCol, value) - def setPredictionCol(value: String): this.type = set(predictionCol, value) - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = false) + private val margin: Vector => Double = (features) => { + BLAS.dot(features, weights) + intercept + } + + private val score: Vector => Double = (features) => { + val m = margin(features) + 1.0 / (1.0 + math.exp(-m)) } override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { @@ -144,4 +132,24 @@ class LogisticRegressionModel private[ml] ( .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol))) } + + override val numClasses: Int = 2 + + /** + * Predict label for the given feature vector. + * The behavior of this can be adjusted using [[threshold]]. + */ + override def predict(features: Vector): Double = { + if (score(features) > paramMap(threshold)) 1 else 0 + } + + override def predictProbabilities(features: Vector): Vector = { + val s = score(features) + Vectors.dense(Array(1.0 - s, s)) + } + + override def predictRaw(features: Vector): Vector = { + val m = margin(m) + Vectors.dense(Array(-m, m)) + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala new file mode 100644 index 0000000000000..fb88bb387d15a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -0,0 +1,67 @@ +package org.apache.spark.ml.classification + +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.ml.param.{HasSmoothingParam, ParamMap} +import org.apache.spark.mllib.linalg.{BLAS, DenseVector, Matrices, Matrix, Vector, Vectors} +import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} +import org.apache.spark.rdd.RDD + + +private[classification] trait NaiveBayesParams extends ClassifierParams with HasSmoothingParam { + // TODO: override validateAndTransformSchema to check smoothingParam validity +} + +class NaiveBayes extends Classifier[NaiveBayes, NaiveBayesModel] with NaiveBayesParams { + + setSmoothingParam(1.0) + + def setSmoothingParam(value: Double): this.type = set(smoothingParam, value) + + override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): NaiveBayesModel = { + val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => + org.apache.spark.mllib.regression.LabeledPoint(label, features) + } + val nb = OldNaiveBayes.train(oldDataset, paramMap(smoothingParam)) + val numClasses = nb.theta.size + val numFeatures = nb.theta(0).size + // Arrange theta into column-major format. + val thetaData = new Array[Double](numClasses * numFeatures) + var j = 0 + var k = 0 // index into thetaData + while (j < numFeatures) { + var i = 0 + while (i < numClasses) { + thetaData(k) = nb.theta(i)(j) + i += 1 + k += 1 + } + j += 1 + } + val theta: Matrix = Matrices.dense(numClasses, numFeatures, thetaData) + new NaiveBayesModel(this, paramMap, new DenseVector(nb.pi), theta) + } +} + +// TODO: Extend ProbabilisticClassificationModel +// TODO: I removed 'labels' since that functionality should be in Classifier. +/** + * @param pi log of class priors, whose dimension is C, number of labels + * @param theta log of class conditional probabilities, whose dimension is C-by-D, + * where D is number of features + */ +class NaiveBayesModel private[ml] ( + override val parent: NaiveBayes, + override val fittingParamMap: ParamMap, + val pi: DenseVector, + val theta: Matrix) + extends ClassificationModel[NaiveBayesModel] with NaiveBayesParams { + + override val numClasses: Int = pi.size + + override def predictRaw(features: Vector): Vector = { + // TODO: Generalize BLAS.gemv to take Vector (not just DenseVector). + val pred = theta.multiply(new DenseVector(features.toArray)) + BLAS.axpy(1.0, pi, pred) + pred + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 1979ab9eb6516..8b51d9d387983 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -21,6 +21,7 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType @@ -56,8 +57,16 @@ class BinaryClassificationEvaluator extends Evaluator with Params .map { case Row(score: Double, label: Double) => (score, label) } + BinaryClassificationEvaluator.computeMetric(scoreAndLabels, map(metricName)) + } + +} + +private[ml] object BinaryClassificationEvaluator { + + def computeMetric(scoreAndLabels: RDD[(Double, Double)], metricName: String): Double = { val metrics = new BinaryClassificationMetrics(scoreAndLabels) - val metric = map(metricName) match { + val metric = metricName match { case "areaUnderROC" => metrics.areaUnderROC() case "areaUnderPR" => @@ -68,4 +77,5 @@ class BinaryClassificationEvaluator extends Evaluator with Params metrics.unpersist() metric } + } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala new file mode 100644 index 0000000000000..be204832cf514 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala @@ -0,0 +1,57 @@ +/* + * 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.evaluation + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.evaluation.impl.PredictionEvaluator +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.evaluation.MulticlassMetrics +import org.apache.spark.rdd.RDD + +/** + * :: AlphaComponent :: + * Evaluator for single-label multiclass classification, + * which expects two input columns: prediction and label. + */ +@AlphaComponent +class ClassificationEvaluator extends PredictionEvaluator { + + /** param for metric name in evaluation */ + val metricName: Param[String] = new Param(this, "metricName", + "metric name in evaluation (accuracy)", Some("accuracy")) + def getMetricName: String = get(metricName) + def setMetricName(value: String): this.type = set(metricName, value) + + protected override def evaluateImpl(predictionsAndLabels: RDD[(Double, Double)]): Double = { + val map = this.paramMap ++ paramMap + ClassificationEvaluator.computeMetric(predictionsAndLabels, map(metricName)) + } +} + +private[ml] object ClassificationEvaluator { + + def computeMetric(predictionsAndLabels: RDD[(Double, Double)], metricName: String): Double = { + val metrics = new MulticlassMetrics(predictionsAndLabels) + metricName match { + case "accuracy" => + metrics.precision + case other => + throw new IllegalArgumentException(s"Does not support metric $other.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala new file mode 100644 index 0000000000000..b82ad4c0d02b3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -0,0 +1,57 @@ +/* + * 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.evaluation + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.evaluation.impl.PredictionEvaluator +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.evaluation.RegressionMetrics +import org.apache.spark.rdd.RDD + +/** + * :: AlphaComponent :: + * Evaluator for single-label regression, + * which expects two input columns: prediction and label. + */ +@AlphaComponent +class RegressionEvaluator extends PredictionEvaluator { + + /** param for metric name in evaluation */ + val metricName: Param[String] = new Param(this, "metricName", + "metric name in evaluation (RMSE)", Some("RMSE")) + def getMetricName: String = get(metricName) + def setMetricName(value: String): this.type = set(metricName, value) + + protected override def evaluateImpl(predictionsAndLabels: RDD[(Double, Double)]): Double = { + val map = this.paramMap ++ paramMap + RegressionEvaluator.computeMetric(predictionsAndLabels, map(metricName)) + } +} + +private[ml] object RegressionEvaluator { + + def computeMetric(predictionsAndLabels: RDD[(Double, Double)], metricName: String): Double = { + val metrics = new RegressionMetrics(predictionsAndLabels) + metricName match { + case "RMSE" => + metrics.rootMeanSquaredError + case other => + throw new IllegalArgumentException(s"Does not support metric $other.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala new file mode 100644 index 0000000000000..bda5351ea672f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala @@ -0,0 +1,59 @@ +/* + * 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.evaluation.impl + +import org.apache.spark.ml._ +import org.apache.spark.ml.param._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DoubleType, Row, SchemaRDD} + +/** + * Evaluator for single-label prediction problems, + * which expects two input columns: prediction and label. + */ +private[ml] abstract class PredictionEvaluator extends Evaluator with Params + with HasPredictionCol with HasLabelCol { + + def setPredictionCol(value: String): this.type = set(predictionCol, value) + def setLabelCol(value: String): this.type = set(labelCol, value) + + override def evaluate(dataset: SchemaRDD, paramMap: ParamMap): Double = { + val map = this.paramMap ++ paramMap + + val schema = dataset.schema + val predictionType = schema(map(predictionCol)).dataType + require(predictionType == DoubleType, + s"Prediction column ${map(predictionCol)} must be double type but found $predictionType") + val labelType = schema(map(labelCol)).dataType + require(labelType == DoubleType, + s"Label column ${map(labelCol)} must be double type but found $labelType") + + import dataset.sqlContext._ + val predictionsAndLabels = dataset.select(map(predictionCol).attr, map(labelCol).attr) + .map { case Row(prediction: Double, label: Double) => + (prediction, label) + } + + evaluateImpl(predictionsAndLabels) + } + + /** + * Developers can implement this method for evaluators taking (prediction, label) tuples + */ + protected def evaluateImpl(predictionsAndLabels: RDD[(Double, Double)]): Double +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala new file mode 100644 index 0000000000000..af57e5e4a2a3e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala @@ -0,0 +1,12 @@ +package org.apache.spark.ml.impl.estimator + +import org.apache.spark.ml.Evaluator + +trait HasDefaultEvaluator { + + /** + * Default evaluation metric usable for model validation (e.g., with CrossValidator). + */ + def defaultEvaluator: Evaluator + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala new file mode 100644 index 0000000000000..484017b786fbd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala @@ -0,0 +1,18 @@ +package org.apache.spark.ml.impl.estimator + +import org.apache.spark.sql.SchemaRDD + +import org.apache.spark.ml.Model +import org.apache.spark.ml.param.ParamMap + +/** + * Trait for an iterative estimator which permits access to its underlying iterative optimization + * algorithm. + * Classes implementing this trait can create an [[IterativeSolver]], which operates on a static + * dataset using an iterative algorithm. + */ +trait IterativeEstimator[M <: Model[M]] { + + private[ml] def createSolver(dataset: SchemaRDD, paramMap: ParamMap): IterativeSolver[M] + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala new file mode 100644 index 0000000000000..b3c2c987da912 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala @@ -0,0 +1,54 @@ +package org.apache.spark.ml.impl.estimator + +import org.apache.spark.ml.Model + +/** + * Iterative stateful solver for an [[IterativeEstimator]]. + * This type of estimator holds state (a fixed dataset and a model) and allows for iterative + * optimization. + * + * This type is distinct from an Optimizer in that an Optimizer has no concept of + * a [[Model]]; an [[IterativeSolver]] can produce a [[Model]] at any time. + * + * This type is not an [[org.apache.spark.ml.Estimator]], but it can produce a model. + * + * Notes to developers: + * - This class allows an algorithm such as LinearRegression to produce an IterativeSolver + * even when the underlying optimization is non-iterative (such as matrix inversion). + * In that case, the step() method can be called once, after which it will do nothing. + */ +abstract class IterativeSolver[M <: Model[M]] { + + protected var currentIteration: Int = 0 + + /** + * Run one step (iteration) of learning. + * @return True if the step completed. + * False if learning had already finished, or if the step failed. + */ + def step(): Boolean = { + // This handles the iteration limit. Developers can implement stepImpl(). + if (currentIteration >= maxIterations) return false + val res = stepImpl() + if (res) currentIteration += 1 + res + } + + /** + * Same as step, except that it ignores the iteration limit. + */ + protected def stepImpl(): Boolean + + /** + * Get the current model. + */ + def currentModel: M + + def maxIterations: Int + + /** + * Number of iterations completed so far. + * If [[step()]] returns false, this count will not be incremented. + */ + def numIterations: Int = currentIteration +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala new file mode 100644 index 0000000000000..89e82c36dd52e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -0,0 +1,135 @@ +package org.apache.spark.ml.impl.estimator + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, LabeledPoint, Model} +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.Star + +@AlphaComponent +private[ml] trait PredictorParams extends Params + with HasLabelCol with HasFeaturesCol with HasPredictionCol { + + /** + * Validates and transforms the input schema with the provided param map. + * @param schema input schema + * @param paramMap additional parameters + * @param fitting whether this is in fitting + * @return output schema + */ + protected def validateAndTransformSchema( + schema: StructType, + paramMap: ParamMap, + fitting: Boolean): StructType = { + val map = this.paramMap ++ paramMap + val featuresType = schema(map(featuresCol)).dataType + // TODO: Support casting Array[Double] and Array[Float] to Vector. + require(featuresType.isInstanceOf[VectorUDT], + s"Features column ${map(featuresCol)} must be Vector types" + + s" but was actually $featuresType.") + if (fitting) { + val labelType = schema(map(labelCol)).dataType + require(labelType == DoubleType || labelType == IntegerType, + s"Cannot convert label column ${map(labelCol)} of type $labelType to a Double column.") + } + val fieldNames = schema.fieldNames + require(!fieldNames.contains(map(predictionCol)), + s"Prediction column ${map(predictionCol)} already exists.") + val outputFields = schema.fields ++ Seq( + StructField(map(predictionCol), DoubleType, nullable = false)) + StructType(outputFields) + } +} + +private[ml] abstract class Predictor[Learner <: Predictor[Learner, M], M <: PredictionModel[M]] + extends Estimator[M] with PredictorParams { + + // TODO: Eliminate asInstanceOf and see if that works. + def setLabelCol(value: String): Learner = set(labelCol, value).asInstanceOf[Learner] + def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] + def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] + + protected def selectLabelColumn(dataset: SchemaRDD, paramMap: ParamMap): RDD[Double] = { + import dataset.sqlContext._ + val map = this.paramMap ++ paramMap + dataset.select(map(labelCol).attr).map { + case Row(label: Double) => label + case Row(label: Int) => label.toDouble + } + } + + private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap, fitting = true) + } + + override def fit(dataset: SchemaRDD, paramMap: ParamMap): M = { + transformSchema(dataset.schema, paramMap, logging = true) + import dataset.sqlContext._ + val map = this.paramMap ++ paramMap + val instances = dataset.select(map(labelCol).attr, map(featuresCol).attr) + .map { case Row(label: Double, features: Vector) => + LabeledPoint(label, features) + } + val model = train(instances, map) + // copy model params + Params.inheritValues(map, this, model) + model + } + + /** + * Notes to developers: + * - Unlike [[fit()]], this method takes [[paramMap]] which has already been + * combined with the internal paramMap. + * - This should handle caching the dataset if needed. + * @param dataset Training data + * @param paramMap Parameters for training. + */ + def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): M +} + +private[ml] abstract class PredictionModel[M <: PredictionModel[M]] + extends Model[M] with PredictorParams { + + def setFeaturesCol(value: String): M = set(featuresCol, value).asInstanceOf[M] + + def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] + + private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap, fitting = false) + } + + /** + * Transforms dataset by reading from [[featuresCol]], calling [[predict( )]], and storing + * the predictions as a new column [[predictionCol]]. + * This default implementation should be overridden as needed. + * @param dataset input dataset + * @param paramMap additional parameters, overwrite embedded params + * @return transformed dataset with [[predictionCol]] of type [[Double]] + */ + override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + import org.apache.spark.sql.catalyst.dsl._ + import dataset.sqlContext._ + + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + val pred: Vector => Double = (features) => { + predict(features) + } + dataset.select(Star(None), pred.call(map(featuresCol).attr) as map(predictionCol)) + } + + /** + * Default implementation. + * Override for efficiency; e.g., this does not broadcast the model. + */ + def predict(dataset: RDD[Vector]): RDD[Double] = { + dataset.map(predict) + } + + /** + * Predict label for the given features. + */ + def predict(features: Vector): Double +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala new file mode 100644 index 0000000000000..a44d4c10a78a4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala @@ -0,0 +1,12 @@ +package org.apache.spark.ml.impl.estimator + +import org.apache.spark.mllib.linalg.Vector + +private[ml] trait ProbabilisticClassificationModel { + + /** + * Predict the probability of each label. + */ + def predictProbabilities(features: Vector): Vector + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala new file mode 100644 index 0000000000000..e9e7628f20f22 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala @@ -0,0 +1,19 @@ +package org.apache.spark.ml.impl.estimator + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SchemaRDD +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.ml.param.ParamMap + +/** + * Trait indicating that this algorithm is optimized for being used as a weak learner for boosting, + * bagging, and other meta-algorithms. + */ +trait WeakLearner[M <: PredictionModel[M]] { + + def getNativeFeatureRDD(dataset: SchemaRDD, paramMap: ParamMap): RDD[Vector] + + def trainNative(dataset: RDD[LabeledPoint], paramMap: ParamMap): M + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index ef141d3eb2b06..0e30377d7285a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -23,6 +23,13 @@ private[ml] trait HasRegParam extends Params { def getRegParam: Double = get(regParam) } +private[ml] trait HasSmoothingParam extends Params { + /** param for smoothing parameter */ + val smoothingParam: DoubleParam = + new DoubleParam(this, "smoothingParam", "smoothing parameter") + def getSmoothingParam: Double = get(smoothingParam) +} + private[ml] trait HasMaxIter extends Params { /** param for max number of iterations */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") @@ -72,3 +79,9 @@ private[ml] trait HasOutputCol extends Params { val outputCol: Param[String] = new Param(this, "outputCol", "output column name") def getOutputCol: String = get(outputCol) } + +private[ml] trait HasWeightCol extends Params { + /** param for instance weight column name */ + val weightCol: Param[String] = new Param(this, "weightCol", "instance weight column name") + def getWeightCol: String = get(weightCol) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala new file mode 100644 index 0000000000000..8a6af90857dd1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -0,0 +1,72 @@ +package org.apache.spark.ml.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.ml.param.{ParamMap, HasMaxIter, HasRegParam} +import org.apache.spark.mllib.linalg.{BLAS, Vector} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +/** + * :: AlphaComponent :: + * Params for linear regression. + */ +@AlphaComponent +private[regression] trait LinearRegressionParams extends RegressorParams + with HasRegParam with HasMaxIter + + +/** + * Logistic regression. + */ +class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel] + with LinearRegressionParams { + + // TODO: Extend IterativeEstimator + + setRegParam(0.1) + setMaxIter(100) + + def setRegParam(value: Double): this.type = set(regParam, value) + def setMaxIter(value: Int): this.type = set(maxIter, value) + + def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { + val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => + org.apache.spark.mllib.regression.LabeledPoint(label, features) + } + val handlePersistence = oldDataset.getStorageLevel == StorageLevel.NONE + if (handlePersistence) { + oldDataset.persist(StorageLevel.MEMORY_AND_DISK) + } + val lr = new LinearRegressionWithSGD() + lr.optimizer + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) + val model = lr.run(oldDataset) + val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) + if (handlePersistence) { + oldDataset.unpersist() + } + lrm + } +} + + +/** + * :: AlphaComponent :: + * Model produced by [[LinearRegression]]. + */ +@AlphaComponent +class LinearRegressionModel private[ml] ( + override val parent: LinearRegression, + override val fittingParamMap: ParamMap, + val weights: Vector, + val intercept: Double) + extends RegressionModel[LinearRegressionModel] + with LinearRegressionParams { + + override def predict(features: Vector): Double = { + BLAS.dot(features, weights) + intercept + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala new file mode 100644 index 0000000000000..d2009d8610a1c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -0,0 +1,33 @@ +package org.apache.spark.ml.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.Evaluator +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.impl.estimator.{PredictionModel, HasDefaultEvaluator, Predictor, + PredictorParams} +import org.apache.spark.mllib.linalg.Vector + +@AlphaComponent +private[regression] trait RegressorParams extends PredictorParams + +/** + * Single-label regression + */ +abstract class Regressor[Learner <: Regressor[Learner, M], M <: RegressionModel[M]] + extends Predictor[Learner, M] + with RegressorParams + with HasDefaultEvaluator { + + override def defaultEvaluator: Evaluator = new RegressionEvaluator +} + + +private[ml] abstract class RegressionModel[M <: RegressionModel[M]] + extends PredictionModel[M] with RegressorParams { + + /** + * Predict real-valued label for the given features. + */ + def predict(features: Vector): Double + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 5d51c51346665..ea4301e6467c3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -22,6 +22,7 @@ import com.github.fommil.netlib.F2jBLAS import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ +import org.apache.spark.ml.impl.estimator.HasDefaultEvaluator import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.DataFrame @@ -70,7 +71,18 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP transformSchema(dataset.schema, paramMap, logging = true) val sqlCtx = dataset.sqlContext val est = map(estimator) - val eval = map(evaluator) + val eval = if (map.contains(evaluator)) { + map(evaluator) + } else { + est match { + case e: HasDefaultEvaluator => + e.defaultEvaluator + case _ => + throw new IllegalArgumentException("CrossValidator could not find an evaluator to use." + + s" The estimator $est does not have a defaultEvaluator, so you must specify the" + + s" evaluator parameter for CrossValidator.") + } + } val epm = map(estimatorParamMaps) val numModels = epm.size val metrics = new Array[Double](epm.size) From d35bb5d924a66d7df73f1d97aaa4ada5cebc2c34 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 23 Nov 2014 23:30:31 -0800 Subject: [PATCH 02/30] fixed compilation issues, but have not added tests yet --- .../scala/org/apache/spark/ml/classification/AdaBoost.scala | 5 +++-- .../apache/spark/ml/classification/LogisticRegression.scala | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala index 5517cd95cf42f..2900570e5f3b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala @@ -57,10 +57,11 @@ class AdaBoost extends Classifier[AdaBoost, AdaBoostModel] val featuresColName = paramMap(featuresCol) val wl = paramMap(weakLearner) val featuresRDD: RDD[Vector] = wl match { - case wlTagged: WeakLearner => + case wlTagged: WeakLearner[_] => val wlParamMap = paramMap(weakLearnerParamMap) val wlFeaturesColName = wlParamMap(wl.featuresCol) - val origFeaturesRDD = dataset.select(featuresColName.attr).as(wlFeaturesColName.attr) + // TODO: How do I get this to use the string value of wlFeaturesColName? + val origFeaturesRDD = dataset.select(featuresColName.attr).as('wlFeaturesColName) wlTagged.getNativeFeatureRDD(origFeaturesRDD, wlParamMap) case _ => dataset.select(featuresColName.attr).map { case Row(features: Vector) => features } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 19b384cf04316..20d9671b301fe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -149,7 +149,7 @@ class LogisticRegressionModel private[ml] ( } override def predictRaw(features: Vector): Vector = { - val m = margin(m) + val m = margin(features) Vectors.dense(Array(-m, m)) } } From 52f4fdec7060ab992fefc9603128a4062bc4ee38 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 1 Dec 2014 10:31:41 -0800 Subject: [PATCH 03/30] removing everything except for simple class hierarchy for classification --- .../org/apache/spark/ml/LabeledPoint.scala | 6 +- .../spark/ml/classification/AdaBoost.scala | 209 ------------------ .../spark/ml/classification/Classifier.scala | 21 +- .../classification/LogisticRegression.scala | 2 - .../spark/ml/classification/NaiveBayes.scala | 67 ------ .../BinaryClassificationEvaluator.scala | 12 +- .../evaluation/ClassificationEvaluator.scala | 57 ----- .../ml/evaluation/RegressionEvaluator.scala | 57 ----- .../evaluation/impl/PredictionEvaluator.scala | 59 ----- .../impl/estimator/HasDefaultEvaluator.scala | 12 - .../impl/estimator/IterativeEstimator.scala | 18 -- .../ml/impl/estimator/IterativeSolver.scala | 54 ----- .../spark/ml/impl/estimator/WeakLearner.scala | 19 -- .../apache/spark/ml/param/sharedParams.scala | 13 -- .../ml/regression/LinearRegression.scala | 72 ------ .../spark/ml/regression/Regressor.scala | 33 --- .../spark/ml/tuning/CrossValidator.scala | 14 +- 17 files changed, 8 insertions(+), 717 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala index 47a6c71b78916..5c5bdfbbd80c1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala @@ -11,14 +11,12 @@ import org.apache.spark.mllib.linalg.Vector */ case class LabeledPoint(label: Double, features: Vector, weight: Double) { - /** Default constructor which sets instance weight to 1.0 */ - def this(label: Double, features: Vector) = this(label, features, 1.0) - override def toString: String = { "(%s,%s,%s)".format(label, features, weight) } } object LabeledPoint { - def apply(label: Double, features: Vector) = new LabeledPoint(label, features) + /** Constructor which sets instance weight to 1.0 */ + def apply(label: Double, features: Vector) = new LabeledPoint(label, features, 1.0) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala deleted file mode 100644 index 2900570e5f3b0..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/AdaBoost.scala +++ /dev/null @@ -1,209 +0,0 @@ -package org.apache.spark.ml.classification - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.mllib.linalg.{Vectors, Vector} -import org.apache.spark.ml.LabeledPoint -import org.apache.spark.ml.evaluation.ClassificationEvaluator -import org.apache.spark.ml.param.{HasWeightCol, Param, ParamMap, HasMaxIter} -import org.apache.spark.ml.impl.estimator.{ProbabilisticClassificationModel, WeakLearner, - IterativeEstimator, IterativeSolver} - - -private[classification] trait AdaBoostParams extends ClassifierParams - with HasMaxIter with HasWeightCol { - - /** param for weak learner type */ - val weakLearner: Param[Classifier[_, _]] = - new Param(this, "weakLearner", "weak learning algorithm") - def getWeakLearner: Classifier[_, _] = get(weakLearner) - - /** param for weak learner param maps */ - val weakLearnerParamMap: Param[ParamMap] = - new Param(this, "weakLearnerParamMap", "param map for the weak learner") - def getWeakLearnerParamMap: ParamMap = get(weakLearnerParamMap) - - override def validate(paramMap: ParamMap): Unit = { - // TODO: Check maxIter, weakLearner, weakLearnerParamMap, weightCol - // Check: If the weak learner does not extend WeakLearner, then featuresColName should be - // castable to FeaturesType. - } -} - - -/** - * AdaBoost - * - * Developer notes: - * - If the weak learner implements the [[WeakLearner]] - */ -class AdaBoost extends Classifier[AdaBoost, AdaBoostModel] - with AdaBoostParams - with IterativeEstimator[AdaBoostModel] { - - def setMaxIter(value: Int): this.type = set(maxIter, value) - def setWeightCol(value: String): this.type = set(weightCol, value) - def setWeakLearner(value: Classifier[_, _]): this.type = set(weakLearner, value) - def setWeakLearnerParamMap(value: ParamMap): this.type = set(weakLearnerParamMap, value) - - /** - * Extract LabeledPoints, using the weak learner's native feature representation if possible. - * @param paramMap Complete paramMap (after combining with the internal paramMap) - */ - private def extractLabeledPoints(dataset: SchemaRDD, paramMap: ParamMap): RDD[LabeledPoint] = { - import dataset.sqlContext._ - val featuresColName = paramMap(featuresCol) - val wl = paramMap(weakLearner) - val featuresRDD: RDD[Vector] = wl match { - case wlTagged: WeakLearner[_] => - val wlParamMap = paramMap(weakLearnerParamMap) - val wlFeaturesColName = wlParamMap(wl.featuresCol) - // TODO: How do I get this to use the string value of wlFeaturesColName? - val origFeaturesRDD = dataset.select(featuresColName.attr).as('wlFeaturesColName) - wlTagged.getNativeFeatureRDD(origFeaturesRDD, wlParamMap) - case _ => - dataset.select(featuresColName.attr).map { case Row(features: Vector) => features } - } - - val labelColName = paramMap(labelCol) - if (paramMap.contains(weightCol)) { - val weightColName = paramMap(weightCol) - dataset.select(labelColName.attr, weightColName.attr) - .zip(featuresRDD).map { case (Row(label: Double, weight: Double), features: Vector) => - LabeledPoint(label, features, weight) - } - } else { - dataset.select(labelColName.attr) - .zip(featuresRDD).map { case (Row(label: Double), features: Vector) => - LabeledPoint(label, features) - } - } - } - - // From Classifier - override def fit(dataset: SchemaRDD, paramMap: ParamMap): AdaBoostModel = { - val map = this.paramMap ++ paramMap - val labeledPoints: RDD[LabeledPoint] = extractLabeledPoints(dataset, map) - train(labeledPoints, paramMap) - } - - // From IterativeEstimator - override private[ml] def createSolver(dataset: SchemaRDD, paramMap: ParamMap): AdaBoostSolver = { - val map = this.paramMap ++ paramMap - val labeledPoints: RDD[LabeledPoint] = extractLabeledPoints(dataset, map) - new AdaBoostSolver(labeledPoints, this, map) - } - - // From Predictor - override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): AdaBoostModel = { - val map = this.paramMap ++ paramMap - val solver = new AdaBoostSolver(dataset, this, map) - while (solver.step()) { } - solver.currentModel - } -} - - -class AdaBoostModel private[ml] ( - val weakHypotheses: Array[ClassificationModel[_]], - val weakHypothesisWeights: Array[Double], - override val parent: AdaBoost, - override val fittingParamMap: ParamMap) - extends ClassificationModel[AdaBoostModel] - with ProbabilisticClassificationModel - with AdaBoostParams { - - require(weakHypotheses.size != 0) - require(weakHypotheses.size == weakHypothesisWeights.size) - - // From Classifier.Model: - override val numClasses: Int = weakHypotheses(0).numClasses - - require(weakHypotheses.forall(_.numClasses == numClasses)) - - private val margin: Vector => Double = (features) => { - weakHypotheses.zip(weakHypothesisWeights) - .foldLeft(0.0) { case (total: Double, (wh: ClassificationModel[_], weight: Double)) => - val pred = if (wh.predict(features) == 1.0) 1.0 else -1.0 - total + weight * pred - } - } - - private val score: Vector => Double = (features) => { - val m = margin(features) - 1.0 / (1.0 + math.exp(-2.0 * m)) - } - - override def predictProbabilities(features: Vector): Vector = { - val s = score(features) - Vectors.dense(Array(1.0 - s, s)) - } - - override def predictRaw(features: Vector): Vector = { - val m = margin(features) - Vectors.dense(Array(-m, m)) - } -} - - -private[ml] class AdaBoostSolver( - val origData: RDD[LabeledPoint], - val parent: AdaBoost, - val paramMap: ParamMap) extends IterativeSolver[AdaBoostModel] { - - private val weakHypotheses = new ArrayBuffer[ClassificationModel[_]] - private val weakHypothesisWeights = new ArrayBuffer[Double] - - private val wl: Classifier[_, _] = paramMap(parent.weakLearner) - private val wlParamMap = paramMap(parent.weakLearnerParamMap) - override val maxIterations: Int = paramMap(parent.maxIter) - - // TODO: Decide if this alg should cache data, or if that should be left to the user. - - // TODO: check for weights = 0 - // TODO: EDITING HERE NOW: switch to log weights - private var logInstanceWeights: RDD[Double] = origData.map(lp => math.log(lp.weight)) - - override def stepImpl(): Boolean = ??? /*{ - // Check if the weak learner takes instance weights. - val wlDataset = wl match { - case wlWeighted: HasWeightCol => - origData.zip(logInstanceWeights).map { case (lp: LabeledPoint, logWeight: Double) => - LabeledPoint(lp.label, lp.features, weight) - } - case _ => - // Subsample data to simulate the current instance weight distribution. - // TODO: This needs to be done before AdaBoost is committed. - throw new NotImplementedError( - "AdaBoost currently requires that the weak learning algorithm accept instance weights.") - } - // Train the weak learning algorithm. - val weakHypothesis: ClassificationModel[_] = wl match { - case wlTagged: WeakLearner[_] => - // This lets the weak learner know that the features are in its native format. - wlTagged.trainNative(wlDataset, wlParamMap).asInstanceOf[ClassificationModel[_]] - case _ => - wl.train(wlDataset, wlParamMap).asInstanceOf[ClassificationModel[_]] - } - // Add the weighted weak hypothesis to the ensemble. - // TODO: Handle instance weights. - val predictionsAndLabels = wlDataset.map(lp => weakHypothesis.predict(lp.features)) - .zip(wlDataset.map(_.label)) - val eps = ClassificationEvaluator.computeMetric(predictionsAndLabels, "accuracy") - val alpha = 0.5 * (math.log(1.0 - eps) - math.log(eps)) // TODO: handle eps near 0 - weakHypotheses += weakHypothesis - weakHypothesisWeights += alpha - // Update weights. - val newInstanceWeights = instanceWeights.zip(predictionsAndLabels).map { - case (weight: Double, (pred: Double, label: Double)) => - ??? - } - - }*/ - - override def currentModel: AdaBoostModel = { - new AdaBoostModel(weakHypotheses.toArray, weakHypothesisWeights.toArray, parent, paramMap) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 5d146f6724958..10fdcf8751118 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -18,12 +18,8 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.evaluation.ClassificationEvaluator +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.ml._ -import org.apache.spark.ml.impl.estimator.{HasDefaultEvaluator, PredictionModel, Predictor, - PredictorParams} -import org.apache.spark.rdd.RDD @AlphaComponent private[classification] trait ClassifierParams extends PredictorParams @@ -33,10 +29,9 @@ private[classification] trait ClassifierParams extends PredictorParams */ abstract class Classifier[Learner <: Classifier[Learner, M], M <: ClassificationModel[M]] extends Predictor[Learner, M] - with ClassifierParams - with HasDefaultEvaluator { + with ClassifierParams { - override def defaultEvaluator: Evaluator = new ClassificationEvaluator + // TODO: defaultEvaluator (follow-up PR) } @@ -60,14 +55,6 @@ private[ml] abstract class ClassificationModel[M <: ClassificationModel[M]] */ def predictRaw(features: Vector): Vector - /** - * Compute this model's accuracy on the given dataset. - */ - def accuracy(dataset: RDD[LabeledPoint]): Double = { - // TODO: Handle instance weights. - val predictionsAndLabels = dataset.map(lp => predict(lp.features)) - .zip(dataset.map(_.label)) - ClassificationEvaluator.computeMetric(predictionsAndLabels, "accuracy") - } + // TODO: accuracy(dataset: RDD[LabeledPoint]): Double (follow-up PR) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 20d9671b301fe..6ef8bd3ce8c06 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -58,8 +58,6 @@ private[classification] trait LogisticRegressionParams extends ClassifierParams class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { - // TODO: Extend IterativeEstimator - setRegParam(0.1) setMaxIter(100) setThreshold(0.5) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala deleted file mode 100644 index fb88bb387d15a..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ /dev/null @@ -1,67 +0,0 @@ -package org.apache.spark.ml.classification - -import org.apache.spark.ml.LabeledPoint -import org.apache.spark.ml.param.{HasSmoothingParam, ParamMap} -import org.apache.spark.mllib.linalg.{BLAS, DenseVector, Matrices, Matrix, Vector, Vectors} -import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} -import org.apache.spark.rdd.RDD - - -private[classification] trait NaiveBayesParams extends ClassifierParams with HasSmoothingParam { - // TODO: override validateAndTransformSchema to check smoothingParam validity -} - -class NaiveBayes extends Classifier[NaiveBayes, NaiveBayesModel] with NaiveBayesParams { - - setSmoothingParam(1.0) - - def setSmoothingParam(value: Double): this.type = set(smoothingParam, value) - - override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): NaiveBayesModel = { - val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => - org.apache.spark.mllib.regression.LabeledPoint(label, features) - } - val nb = OldNaiveBayes.train(oldDataset, paramMap(smoothingParam)) - val numClasses = nb.theta.size - val numFeatures = nb.theta(0).size - // Arrange theta into column-major format. - val thetaData = new Array[Double](numClasses * numFeatures) - var j = 0 - var k = 0 // index into thetaData - while (j < numFeatures) { - var i = 0 - while (i < numClasses) { - thetaData(k) = nb.theta(i)(j) - i += 1 - k += 1 - } - j += 1 - } - val theta: Matrix = Matrices.dense(numClasses, numFeatures, thetaData) - new NaiveBayesModel(this, paramMap, new DenseVector(nb.pi), theta) - } -} - -// TODO: Extend ProbabilisticClassificationModel -// TODO: I removed 'labels' since that functionality should be in Classifier. -/** - * @param pi log of class priors, whose dimension is C, number of labels - * @param theta log of class conditional probabilities, whose dimension is C-by-D, - * where D is number of features - */ -class NaiveBayesModel private[ml] ( - override val parent: NaiveBayes, - override val fittingParamMap: ParamMap, - val pi: DenseVector, - val theta: Matrix) - extends ClassificationModel[NaiveBayesModel] with NaiveBayesParams { - - override val numClasses: Int = pi.size - - override def predictRaw(features: Vector): Vector = { - // TODO: Generalize BLAS.gemv to take Vector (not just DenseVector). - val pred = theta.multiply(new DenseVector(features.toArray)) - BLAS.axpy(1.0, pi, pred) - pred - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 8b51d9d387983..1979ab9eb6516 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType @@ -57,16 +56,8 @@ class BinaryClassificationEvaluator extends Evaluator with Params .map { case Row(score: Double, label: Double) => (score, label) } - BinaryClassificationEvaluator.computeMetric(scoreAndLabels, map(metricName)) - } - -} - -private[ml] object BinaryClassificationEvaluator { - - def computeMetric(scoreAndLabels: RDD[(Double, Double)], metricName: String): Double = { val metrics = new BinaryClassificationMetrics(scoreAndLabels) - val metric = metricName match { + val metric = map(metricName) match { case "areaUnderROC" => metrics.areaUnderROC() case "areaUnderPR" => @@ -77,5 +68,4 @@ private[ml] object BinaryClassificationEvaluator { metrics.unpersist() metric } - } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala deleted file mode 100644 index be204832cf514..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClassificationEvaluator.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.evaluation - -import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.evaluation.impl.PredictionEvaluator -import org.apache.spark.ml.param._ -import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.rdd.RDD - -/** - * :: AlphaComponent :: - * Evaluator for single-label multiclass classification, - * which expects two input columns: prediction and label. - */ -@AlphaComponent -class ClassificationEvaluator extends PredictionEvaluator { - - /** param for metric name in evaluation */ - val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (accuracy)", Some("accuracy")) - def getMetricName: String = get(metricName) - def setMetricName(value: String): this.type = set(metricName, value) - - protected override def evaluateImpl(predictionsAndLabels: RDD[(Double, Double)]): Double = { - val map = this.paramMap ++ paramMap - ClassificationEvaluator.computeMetric(predictionsAndLabels, map(metricName)) - } -} - -private[ml] object ClassificationEvaluator { - - def computeMetric(predictionsAndLabels: RDD[(Double, Double)], metricName: String): Double = { - val metrics = new MulticlassMetrics(predictionsAndLabels) - metricName match { - case "accuracy" => - metrics.precision - case other => - throw new IllegalArgumentException(s"Does not support metric $other.") - } - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala deleted file mode 100644 index b82ad4c0d02b3..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.evaluation - -import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.evaluation.impl.PredictionEvaluator -import org.apache.spark.ml.param._ -import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.rdd.RDD - -/** - * :: AlphaComponent :: - * Evaluator for single-label regression, - * which expects two input columns: prediction and label. - */ -@AlphaComponent -class RegressionEvaluator extends PredictionEvaluator { - - /** param for metric name in evaluation */ - val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (RMSE)", Some("RMSE")) - def getMetricName: String = get(metricName) - def setMetricName(value: String): this.type = set(metricName, value) - - protected override def evaluateImpl(predictionsAndLabels: RDD[(Double, Double)]): Double = { - val map = this.paramMap ++ paramMap - RegressionEvaluator.computeMetric(predictionsAndLabels, map(metricName)) - } -} - -private[ml] object RegressionEvaluator { - - def computeMetric(predictionsAndLabels: RDD[(Double, Double)], metricName: String): Double = { - val metrics = new RegressionMetrics(predictionsAndLabels) - metricName match { - case "RMSE" => - metrics.rootMeanSquaredError - case other => - throw new IllegalArgumentException(s"Does not support metric $other.") - } - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala deleted file mode 100644 index bda5351ea672f..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/impl/PredictionEvaluator.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.evaluation.impl - -import org.apache.spark.ml._ -import org.apache.spark.ml.param._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DoubleType, Row, SchemaRDD} - -/** - * Evaluator for single-label prediction problems, - * which expects two input columns: prediction and label. - */ -private[ml] abstract class PredictionEvaluator extends Evaluator with Params - with HasPredictionCol with HasLabelCol { - - def setPredictionCol(value: String): this.type = set(predictionCol, value) - def setLabelCol(value: String): this.type = set(labelCol, value) - - override def evaluate(dataset: SchemaRDD, paramMap: ParamMap): Double = { - val map = this.paramMap ++ paramMap - - val schema = dataset.schema - val predictionType = schema(map(predictionCol)).dataType - require(predictionType == DoubleType, - s"Prediction column ${map(predictionCol)} must be double type but found $predictionType") - val labelType = schema(map(labelCol)).dataType - require(labelType == DoubleType, - s"Label column ${map(labelCol)} must be double type but found $labelType") - - import dataset.sqlContext._ - val predictionsAndLabels = dataset.select(map(predictionCol).attr, map(labelCol).attr) - .map { case Row(prediction: Double, label: Double) => - (prediction, label) - } - - evaluateImpl(predictionsAndLabels) - } - - /** - * Developers can implement this method for evaluators taking (prediction, label) tuples - */ - protected def evaluateImpl(predictionsAndLabels: RDD[(Double, Double)]): Double -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala deleted file mode 100644 index af57e5e4a2a3e..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/HasDefaultEvaluator.scala +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.spark.ml.impl.estimator - -import org.apache.spark.ml.Evaluator - -trait HasDefaultEvaluator { - - /** - * Default evaluation metric usable for model validation (e.g., with CrossValidator). - */ - def defaultEvaluator: Evaluator - -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala deleted file mode 100644 index 484017b786fbd..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeEstimator.scala +++ /dev/null @@ -1,18 +0,0 @@ -package org.apache.spark.ml.impl.estimator - -import org.apache.spark.sql.SchemaRDD - -import org.apache.spark.ml.Model -import org.apache.spark.ml.param.ParamMap - -/** - * Trait for an iterative estimator which permits access to its underlying iterative optimization - * algorithm. - * Classes implementing this trait can create an [[IterativeSolver]], which operates on a static - * dataset using an iterative algorithm. - */ -trait IterativeEstimator[M <: Model[M]] { - - private[ml] def createSolver(dataset: SchemaRDD, paramMap: ParamMap): IterativeSolver[M] - -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala deleted file mode 100644 index b3c2c987da912..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/IterativeSolver.scala +++ /dev/null @@ -1,54 +0,0 @@ -package org.apache.spark.ml.impl.estimator - -import org.apache.spark.ml.Model - -/** - * Iterative stateful solver for an [[IterativeEstimator]]. - * This type of estimator holds state (a fixed dataset and a model) and allows for iterative - * optimization. - * - * This type is distinct from an Optimizer in that an Optimizer has no concept of - * a [[Model]]; an [[IterativeSolver]] can produce a [[Model]] at any time. - * - * This type is not an [[org.apache.spark.ml.Estimator]], but it can produce a model. - * - * Notes to developers: - * - This class allows an algorithm such as LinearRegression to produce an IterativeSolver - * even when the underlying optimization is non-iterative (such as matrix inversion). - * In that case, the step() method can be called once, after which it will do nothing. - */ -abstract class IterativeSolver[M <: Model[M]] { - - protected var currentIteration: Int = 0 - - /** - * Run one step (iteration) of learning. - * @return True if the step completed. - * False if learning had already finished, or if the step failed. - */ - def step(): Boolean = { - // This handles the iteration limit. Developers can implement stepImpl(). - if (currentIteration >= maxIterations) return false - val res = stepImpl() - if (res) currentIteration += 1 - res - } - - /** - * Same as step, except that it ignores the iteration limit. - */ - protected def stepImpl(): Boolean - - /** - * Get the current model. - */ - def currentModel: M - - def maxIterations: Int - - /** - * Number of iterations completed so far. - * If [[step()]] returns false, this count will not be incremented. - */ - def numIterations: Int = currentIteration -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala deleted file mode 100644 index e9e7628f20f22..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/WeakLearner.scala +++ /dev/null @@ -1,19 +0,0 @@ -package org.apache.spark.ml.impl.estimator - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SchemaRDD -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.ml.LabeledPoint -import org.apache.spark.ml.param.ParamMap - -/** - * Trait indicating that this algorithm is optimized for being used as a weak learner for boosting, - * bagging, and other meta-algorithms. - */ -trait WeakLearner[M <: PredictionModel[M]] { - - def getNativeFeatureRDD(dataset: SchemaRDD, paramMap: ParamMap): RDD[Vector] - - def trainNative(dataset: RDD[LabeledPoint], paramMap: ParamMap): M - -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index 0e30377d7285a..ef141d3eb2b06 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -23,13 +23,6 @@ private[ml] trait HasRegParam extends Params { def getRegParam: Double = get(regParam) } -private[ml] trait HasSmoothingParam extends Params { - /** param for smoothing parameter */ - val smoothingParam: DoubleParam = - new DoubleParam(this, "smoothingParam", "smoothing parameter") - def getSmoothingParam: Double = get(smoothingParam) -} - private[ml] trait HasMaxIter extends Params { /** param for max number of iterations */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") @@ -79,9 +72,3 @@ private[ml] trait HasOutputCol extends Params { val outputCol: Param[String] = new Param(this, "outputCol", "output column name") def getOutputCol: String = get(outputCol) } - -private[ml] trait HasWeightCol extends Params { - /** param for instance weight column name */ - val weightCol: Param[String] = new Param(this, "weightCol", "instance weight column name") - def getWeightCol: String = get(weightCol) -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala deleted file mode 100644 index 8a6af90857dd1..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ /dev/null @@ -1,72 +0,0 @@ -package org.apache.spark.ml.regression - -import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.LabeledPoint -import org.apache.spark.ml.param.{ParamMap, HasMaxIter, HasRegParam} -import org.apache.spark.mllib.linalg.{BLAS, Vector} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel - -/** - * :: AlphaComponent :: - * Params for linear regression. - */ -@AlphaComponent -private[regression] trait LinearRegressionParams extends RegressorParams - with HasRegParam with HasMaxIter - - -/** - * Logistic regression. - */ -class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel] - with LinearRegressionParams { - - // TODO: Extend IterativeEstimator - - setRegParam(0.1) - setMaxIter(100) - - def setRegParam(value: Double): this.type = set(regParam, value) - def setMaxIter(value: Int): this.type = set(maxIter, value) - - def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { - val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => - org.apache.spark.mllib.regression.LabeledPoint(label, features) - } - val handlePersistence = oldDataset.getStorageLevel == StorageLevel.NONE - if (handlePersistence) { - oldDataset.persist(StorageLevel.MEMORY_AND_DISK) - } - val lr = new LinearRegressionWithSGD() - lr.optimizer - .setRegParam(paramMap(regParam)) - .setNumIterations(paramMap(maxIter)) - val model = lr.run(oldDataset) - val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) - if (handlePersistence) { - oldDataset.unpersist() - } - lrm - } -} - - -/** - * :: AlphaComponent :: - * Model produced by [[LinearRegression]]. - */ -@AlphaComponent -class LinearRegressionModel private[ml] ( - override val parent: LinearRegression, - override val fittingParamMap: ParamMap, - val weights: Vector, - val intercept: Double) - extends RegressionModel[LinearRegressionModel] - with LinearRegressionParams { - - override def predict(features: Vector): Double = { - BLAS.dot(features, weights) + intercept - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala deleted file mode 100644 index d2009d8610a1c..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ /dev/null @@ -1,33 +0,0 @@ -package org.apache.spark.ml.regression - -import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.Evaluator -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.ml.impl.estimator.{PredictionModel, HasDefaultEvaluator, Predictor, - PredictorParams} -import org.apache.spark.mllib.linalg.Vector - -@AlphaComponent -private[regression] trait RegressorParams extends PredictorParams - -/** - * Single-label regression - */ -abstract class Regressor[Learner <: Regressor[Learner, M], M <: RegressionModel[M]] - extends Predictor[Learner, M] - with RegressorParams - with HasDefaultEvaluator { - - override def defaultEvaluator: Evaluator = new RegressionEvaluator -} - - -private[ml] abstract class RegressionModel[M <: RegressionModel[M]] - extends PredictionModel[M] with RegressorParams { - - /** - * Predict real-valued label for the given features. - */ - def predict(features: Vector): Double - -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index ea4301e6467c3..5d51c51346665 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -22,7 +22,6 @@ import com.github.fommil.netlib.F2jBLAS import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ -import org.apache.spark.ml.impl.estimator.HasDefaultEvaluator import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.DataFrame @@ -71,18 +70,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP transformSchema(dataset.schema, paramMap, logging = true) val sqlCtx = dataset.sqlContext val est = map(estimator) - val eval = if (map.contains(evaluator)) { - map(evaluator) - } else { - est match { - case e: HasDefaultEvaluator => - e.defaultEvaluator - case _ => - throw new IllegalArgumentException("CrossValidator could not find an evaluator to use." + - s" The estimator $est does not have a defaultEvaluator, so you must specify the" + - s" evaluator parameter for CrossValidator.") - } - } + val eval = map(evaluator) val epm = map(estimatorParamMaps) val numModels = epm.size val metrics = new Array[Double](epm.size) From d705e8751fbc92d30508febb2b57fdd99751451c Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 13:37:29 -0800 Subject: [PATCH 04/30] Added LinearRegression and Regressor back from ml-api branch --- .../ml/regression/LinearRegression.scala | 72 +++++++++++++++++++ .../spark/ml/regression/Regressor.scala | 33 +++++++++ 2 files changed, 105 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala new file mode 100644 index 0000000000000..8a6af90857dd1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -0,0 +1,72 @@ +package org.apache.spark.ml.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.ml.param.{ParamMap, HasMaxIter, HasRegParam} +import org.apache.spark.mllib.linalg.{BLAS, Vector} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +/** + * :: AlphaComponent :: + * Params for linear regression. + */ +@AlphaComponent +private[regression] trait LinearRegressionParams extends RegressorParams + with HasRegParam with HasMaxIter + + +/** + * Logistic regression. + */ +class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel] + with LinearRegressionParams { + + // TODO: Extend IterativeEstimator + + setRegParam(0.1) + setMaxIter(100) + + def setRegParam(value: Double): this.type = set(regParam, value) + def setMaxIter(value: Int): this.type = set(maxIter, value) + + def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { + val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => + org.apache.spark.mllib.regression.LabeledPoint(label, features) + } + val handlePersistence = oldDataset.getStorageLevel == StorageLevel.NONE + if (handlePersistence) { + oldDataset.persist(StorageLevel.MEMORY_AND_DISK) + } + val lr = new LinearRegressionWithSGD() + lr.optimizer + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) + val model = lr.run(oldDataset) + val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) + if (handlePersistence) { + oldDataset.unpersist() + } + lrm + } +} + + +/** + * :: AlphaComponent :: + * Model produced by [[LinearRegression]]. + */ +@AlphaComponent +class LinearRegressionModel private[ml] ( + override val parent: LinearRegression, + override val fittingParamMap: ParamMap, + val weights: Vector, + val intercept: Double) + extends RegressionModel[LinearRegressionModel] + with LinearRegressionParams { + + override def predict(features: Vector): Double = { + BLAS.dot(features, weights) + intercept + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala new file mode 100644 index 0000000000000..d2009d8610a1c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -0,0 +1,33 @@ +package org.apache.spark.ml.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.Evaluator +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.impl.estimator.{PredictionModel, HasDefaultEvaluator, Predictor, + PredictorParams} +import org.apache.spark.mllib.linalg.Vector + +@AlphaComponent +private[regression] trait RegressorParams extends PredictorParams + +/** + * Single-label regression + */ +abstract class Regressor[Learner <: Regressor[Learner, M], M <: RegressionModel[M]] + extends Predictor[Learner, M] + with RegressorParams + with HasDefaultEvaluator { + + override def defaultEvaluator: Evaluator = new RegressionEvaluator +} + + +private[ml] abstract class RegressionModel[M <: RegressionModel[M]] + extends PredictionModel[M] with RegressorParams { + + /** + * Predict real-valued label for the given features. + */ + def predict(features: Vector): Double + +} From 601e7928199ff1a801b0d18afda50ef19645d0f9 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 16:11:02 -0800 Subject: [PATCH 05/30] Modified ParamMap to sort parameters in toString. Cleaned up classes in class hierarchy, before implementing tests and examples. --- .../org/apache/spark/ml/LabeledPoint.scala | 27 ++++++++++++ .../spark/ml/classification/Classifier.scala | 29 ++++++++++--- .../spark/ml/impl/estimator/Predictor.scala | 19 +++++++- .../ProbabilisticClassificationModel.scala | 27 +++++++++++- .../org/apache/spark/ml/param/params.scala | 2 +- .../ml/regression/LinearRegression.scala | 24 ++++++++--- .../spark/ml/regression/Regressor.scala | 43 ++++++++++++++----- 7 files changed, 146 insertions(+), 25 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala index 5c5bdfbbd80c1..8a310fc7b1fee 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala @@ -1,14 +1,37 @@ +/* + * 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 +import scala.beans.BeanInfo + +import org.apache.spark.annotation.AlphaComponent import org.apache.spark.mllib.linalg.Vector /** + * :: AlphaComponent :: * Class that represents an instance (data point) for prediction tasks. * * @param label Label to predict * @param features List of features describing this instance * @param weight Instance weight */ +@AlphaComponent +@BeanInfo case class LabeledPoint(label: Double, features: Vector, weight: Double) { override def toString: String = { @@ -16,6 +39,10 @@ case class LabeledPoint(label: Double, features: Vector, weight: Double) { } } +/** + * :: AlphaComponent :: + */ +@AlphaComponent object LabeledPoint { /** Constructor which sets instance weight to 1.0 */ def apply(label: Double, features: Vector) = new LabeledPoint(label, features, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 10fdcf8751118..eeef116c876db 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -21,12 +21,17 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.mllib.linalg.Vector -@AlphaComponent +/** + * Params for classification. + * Currently empty, but may add functionality later. + */ private[classification] trait ClassifierParams extends PredictorParams /** * Single-label binary or multiclass classification + * Classes are indexed {0, 1, ..., numClasses - 1}. */ +@AlphaComponent abstract class Classifier[Learner <: Classifier[Learner, M], M <: ClassificationModel[M]] extends Predictor[Learner, M] with ClassifierParams { @@ -34,14 +39,22 @@ abstract class Classifier[Learner <: Classifier[Learner, M], M <: Classification // TODO: defaultEvaluator (follow-up PR) } - -private[ml] abstract class ClassificationModel[M <: ClassificationModel[M]] +/** + * :: AlphaComponent :: + * Model produced by a [[Classifier]]. + * Classes are indexed {0, 1, ..., numClasses - 1}. + * + * @tparam M Model type. + */ +@AlphaComponent +abstract class ClassificationModel[M <: ClassificationModel[M]] extends PredictionModel[M] with ClassifierParams { + /** Number of classes (values which the label can take). */ def numClasses: Int /** - * Predict label for the given features. Labels are indexed {0, 1, ..., numClasses - 1}. + * Predict label for the given features. * This default implementation for classification predicts the index of the maximum value * from [[predictRaw()]]. */ @@ -50,8 +63,12 @@ private[ml] abstract class ClassificationModel[M <: ClassificationModel[M]] } /** - * Raw prediction for each possible label - * @return vector where element i is the raw score for label i + * Raw prediction for each possible label. + * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives + * a magnitude of confidence in each possible label. + * @return vector where element i is the raw prediction for label i. + * This raw prediction may be any real number, where a larger value indicates greater + * confidence for that label. */ def predictRaw(features: Vector): Vector diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 89e82c36dd52e..b8fde6b08ebdb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -1,6 +1,22 @@ +/* + * 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.impl.estimator -import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.{Estimator, LabeledPoint, Model} import org.apache.spark.ml.param._ import org.apache.spark.mllib.linalg.{Vector, VectorUDT} @@ -8,7 +24,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star -@AlphaComponent private[ml] trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala index a44d4c10a78a4..52096ce6b2589 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala @@ -1,11 +1,36 @@ +/* + * 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.impl.estimator import org.apache.spark.mllib.linalg.Vector +/** + * Trait for a [[org.apache.spark.ml.classification.ClassificationModel]] which can output + * class conditional probabilities. + */ private[ml] trait ProbabilisticClassificationModel { /** - * Predict the probability of each label. + * Predict the probability of each class given the features. + * These predictions are also called class conditional probabilities. + * + * WARNING: Not all models output well-calibrated probability estimates! These probabilities + * should be treated as confidences, not precise probabilities. */ def predictProbabilities(features: Vector): Vector diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 5fb4379e23c2f..33cfd9bdc364f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -279,7 +279,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten def copy: ParamMap = new ParamMap(map.clone()) override def toString: String = { - map.map { case (param, value) => + map.toSeq.sorted.map { case (param, value) => s"\t${param.parent.uid}-${param.name}: $value" }.mkString("{\n", ",\n", "\n}") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 8a6af90857dd1..0a95acb8a74b5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -1,3 +1,20 @@ +/* + * 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.regression import org.apache.spark.annotation.AlphaComponent @@ -9,22 +26,20 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** - * :: AlphaComponent :: * Params for linear regression. */ @AlphaComponent private[regression] trait LinearRegressionParams extends RegressorParams with HasRegParam with HasMaxIter - /** + * :: AlphaComponent :: * Logistic regression. */ +@AlphaComponent class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel] with LinearRegressionParams { - // TODO: Extend IterativeEstimator - setRegParam(0.1) setMaxIter(100) @@ -52,7 +67,6 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel } } - /** * :: AlphaComponent :: * Model produced by [[LinearRegression]]. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index d2009d8610a1c..78086fe16fd60 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -1,28 +1,51 @@ +/* + * 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.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.Evaluator -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.ml.impl.estimator.{PredictionModel, HasDefaultEvaluator, Predictor, - PredictorParams} +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.mllib.linalg.Vector -@AlphaComponent +/** + * Params for regression. + * Currently empty, but may add functionality later. + */ private[regression] trait RegressorParams extends PredictorParams /** + * :: AlphaComponent :: * Single-label regression */ +@AlphaComponent abstract class Regressor[Learner <: Regressor[Learner, M], M <: RegressionModel[M]] extends Predictor[Learner, M] - with RegressorParams - with HasDefaultEvaluator { + with RegressorParams { - override def defaultEvaluator: Evaluator = new RegressionEvaluator + // TODO: defaultEvaluator (follow-up PR) } - -private[ml] abstract class RegressionModel[M <: RegressionModel[M]] +/** + * :: AlphaComponent :: + * Model produced by a [[Regressor]]. + * @tparam M Model type. + */ +@AlphaComponent +abstract class RegressionModel[M <: RegressionModel[M]] extends PredictionModel[M] with RegressorParams { /** From 0617d61a9b4f6927e4341564c2c274ba5844fec1 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 17:11:22 -0800 Subject: [PATCH 06/30] Fixed bug from last commit (sorting paramMap by parameter names in toString). Fixed bug in persisting logreg data. Added threshold_internal to logreg for faster test-time prediction (avoiding map lookup). --- .../classification/LogisticRegression.scala | 22 ++++++++++++------- .../org/apache/spark/ml/param/params.scala | 2 +- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 6ef8bd3ce8c06..e64041ef2abad 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -54,6 +54,7 @@ private[classification] trait LogisticRegressionParams extends ClassifierParams /** * Logistic regression. + * Currently, this class only supports binary classification. */ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { @@ -71,7 +72,8 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) } - val handlePersistence = oldDataset.getStorageLevel == StorageLevel.NONE + // If dataset is persisted, do not persist oldDataset. + val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) } @@ -84,6 +86,7 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi if (handlePersistence) { oldDataset.unpersist() } + lrm.setThreshold(paramMap(threshold)) lrm } } @@ -103,9 +106,15 @@ class LogisticRegressionModel private[ml] ( with ProbabilisticClassificationModel with LogisticRegressionParams { - def setThreshold(value: Double): this.type = set(threshold, value) + def setThreshold(value: Double): this.type = { + this.threshold_internal = value + set(threshold, value) + } def setScoreCol(value: String): this.type = set(scoreCol, value) + /** Store for faster test-time prediction. */ + private var threshold_internal: Double = this.getThreshold + private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept } @@ -121,11 +130,8 @@ class LogisticRegressionModel private[ml] ( val scoreFunction = udf { v: Vector => val margin = BLAS.dot(v, weights) 1.0 / (1.0 + math.exp(-margin)) - } - val t = map(threshold) - val predictFunction = udf { score: Double => - if (score > t) 1.0 else 0.0 - } + val t = threshold_internal + val predictFunction: Double => Double = (score) => { if (score > t) 1.0 else 0.0 } dataset .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol))) @@ -138,7 +144,7 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override def predict(features: Vector): Double = { - if (score(features) > paramMap(threshold)) 1 else 0 + if (score(features) > threshold_internal) 1 else 0 } override def predictProbabilities(features: Vector): Vector = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 33cfd9bdc364f..465bfa9099c1d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -279,7 +279,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten def copy: ParamMap = new ParamMap(map.clone()) override def toString: String = { - map.toSeq.sorted.map { case (param, value) => + map.toSeq.sortBy(_._1.name).map { case (param, value) => s"\t${param.parent.uid}-${param.name}: $value" }.mkString("{\n", ",\n", "\n}") } From 54b7b311feb17025f96e779b6ee3864b6f1c7177 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 5 Dec 2014 10:22:10 -0800 Subject: [PATCH 07/30] Fixed issue with logreg threshold being set correctly --- .../ml/classification/LogisticRegression.scala | 13 +++++++++---- .../spark/ml/regression/LinearRegression.scala | 1 - 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index e64041ef2abad..0ba2ef6936a17 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -30,10 +30,8 @@ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} import org.apache.spark.storage.StorageLevel /** - * :: AlphaComponent :: * Params for logistic regression. */ -@AlphaComponent private[classification] trait LogisticRegressionParams extends ClassifierParams with HasRegParam with HasMaxIter with HasThreshold with HasScoreCol { @@ -53,9 +51,11 @@ private[classification] trait LogisticRegressionParams extends ClassifierParams /** + * :: AlphaComponent :: * Logistic regression. * Currently, this class only supports binary classification. */ +@AlphaComponent class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { @@ -106,14 +106,19 @@ class LogisticRegressionModel private[ml] ( with ProbabilisticClassificationModel with LogisticRegressionParams { + setThreshold(0.5) + def setThreshold(value: Double): this.type = { this.threshold_internal = value set(threshold, value) } def setScoreCol(value: String): this.type = set(scoreCol, value) - /** Store for faster test-time prediction. */ - private var threshold_internal: Double = this.getThreshold + /** + * Store for faster test-time prediction. + * Initialized to threshold in fittingParamMap if exists, else default threshold. + */ + private var threshold_internal: Double = fittingParamMap.get(threshold).getOrElse(getThreshold) private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 0a95acb8a74b5..a61a02aac61d1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -28,7 +28,6 @@ import org.apache.spark.storage.StorageLevel /** * Params for linear regression. */ -@AlphaComponent private[regression] trait LinearRegressionParams extends RegressorParams with HasRegParam with HasMaxIter From e433872d27776d9a209164a21f266d1691a5fd82 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 5 Dec 2014 10:43:14 -0800 Subject: [PATCH 08/30] Updated docs. Added LabeledPointSuite to spark.ml --- .../scala/org/apache/spark/ml/Estimator.scala | 9 ++- .../classification/LogisticRegression.scala | 7 +++ .../spark/ml/impl/estimator/Predictor.scala | 7 +-- .../ml/regression/LinearRegression.scala | 7 +++ .../apache/spark/ml/LabeledPointSuite.scala | 57 +++++++++++++++++++ 5 files changed, 80 insertions(+), 7 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index bc3defe968afd..eff7ef925dfbd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -34,7 +34,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Fits a single model to the input data with optional parameters. * * @param dataset input dataset - * @param paramPairs optional list of param pairs (overwrite embedded params) + * @param paramPairs Optional list of param pairs. + * These values override any specified in this Estimator's embedded ParamMap. * @return fitted model */ @varargs @@ -47,7 +48,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Fits a single model to the input data with provided parameter map. * * @param dataset input dataset - * @param paramMap parameter map + * @param paramMap Parameter map. + * These values override any specified in this Estimator's embedded ParamMap. * @return fitted model */ def fit(dataset: DataFrame, paramMap: ParamMap): M @@ -58,7 +60,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Subclasses could overwrite this to optimize multi-model training. * * @param dataset input dataset - * @param paramMaps an array of parameter maps + * @param paramMaps An array of parameter maps. + * These values override any specified in this Estimator's embedded ParamMap. * @return fitted models, matching the input parameter maps */ def fit(dataset: DataFrame, paramMaps: Array[ParamMap]): Seq[M] = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 0ba2ef6936a17..aae8372bffcc2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -68,6 +68,13 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi def setThreshold(value: Double): this.type = set(threshold, value) def setScoreCol(value: String): this.type = set(scoreCol, value) + /** + * Same as [[fit()]], but using strong types. + * + * @param dataset Training data. WARNING: This does not yet handle instance weights. + * @param paramMap Parameters for training. + * These values override any specified in this Estimator's embedded ParamMap. + */ def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LogisticRegressionModel = { val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index b8fde6b08ebdb..709ed3a696e49 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -94,12 +94,11 @@ private[ml] abstract class Predictor[Learner <: Predictor[Learner, M], M <: Pred } /** - * Notes to developers: - * - Unlike [[fit()]], this method takes [[paramMap]] which has already been - * combined with the internal paramMap. - * - This should handle caching the dataset if needed. + * Same as [[fit()]], but using strong types. + * * @param dataset Training data * @param paramMap Parameters for training. + * These values override any specified in this Estimator's embedded ParamMap. */ def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): M } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index a61a02aac61d1..c34746e9187da 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -45,6 +45,13 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) + /** + * Same as [[fit()]], but using strong types. + * + * @param dataset Training data. WARNING: This does not yet handle instance weights. + * @param paramMap Parameters for training. + * These values override any specified in this Estimator's embedded ParamMap. + */ def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) diff --git a/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala new file mode 100644 index 0000000000000..34460a9e21d0d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala @@ -0,0 +1,57 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{SQLContext, SchemaRDD} + +class LabeledPointSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("LabeledPoint default weight 1.0") { + val label = 1.0 + val features = Vectors.dense(1.0, 2.0, 3.0) + val lp1 = LabeledPoint(label, features) + val lp2 = LabeledPoint(label, features, weight = 1.0) + assert(lp1 === lp2) + } + + test("Create SchemaRDD from RDD[LabeledPoint]") { + val sqlContext = this.sqlContext + import sqlContext._ + val arr = Seq( + LabeledPoint(0.0, Vectors.dense(1.0, 2.0, 3.0)), + LabeledPoint(1.0, Vectors.dense(1.1, 2.1, 3.1)), + LabeledPoint(0.0, Vectors.dense(1.2, 2.2, 3.2)), + LabeledPoint(1.0, Vectors.dense(1.3, 2.3, 3.3))) + val rdd = sc.parallelize(arr) + val schemaRDD = rdd.select('label, 'features) + val points = schemaRDD.collect() + assert(points.size === arr.size) + } +} From 57d54ab74b5063a2fc7a41868008968ece898ebc Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 5 Dec 2014 15:15:48 -0800 Subject: [PATCH 09/30] * Changed semantics of Predictor.train() to merge the given paramMap with the embedded paramMap. * remove threshold_internal from logreg * Added Predictor.copy() * Extended LogisticRegressionSuite --- .../classification/LogisticRegression.scala | 32 +++++++------- .../spark/ml/impl/estimator/Predictor.scala | 21 +++++++--- .../ml/regression/LinearRegression.scala | 8 +++- .../LogisticRegressionSuite.scala | 42 +++++++++++++++++-- 4 files changed, 78 insertions(+), 25 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index aae8372bffcc2..cfb3a6bda041f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -76,6 +76,7 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi * These values override any specified in this Estimator's embedded ParamMap. */ def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LogisticRegressionModel = { + val map = this.paramMap ++ paramMap val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) } @@ -86,14 +87,13 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi } val lr = new LogisticRegressionWithLBFGS lr.optimizer - .setRegParam(paramMap(regParam)) - .setNumIterations(paramMap(maxIter)) + .setRegParam(map(regParam)) + .setNumIterations(map(maxIter)) val model = lr.run(oldDataset) - val lrm = new LogisticRegressionModel(this, paramMap, model.weights, model.intercept) + val lrm = new LogisticRegressionModel(this, map, model.weights, model.intercept) if (handlePersistence) { oldDataset.unpersist() } - lrm.setThreshold(paramMap(threshold)) lrm } } @@ -115,18 +115,9 @@ class LogisticRegressionModel private[ml] ( setThreshold(0.5) - def setThreshold(value: Double): this.type = { - this.threshold_internal = value - set(threshold, value) - } + def setThreshold(value: Double): this.type = set(threshold, value) def setScoreCol(value: String): this.type = set(scoreCol, value) - /** - * Store for faster test-time prediction. - * Initialized to threshold in fittingParamMap if exists, else default threshold. - */ - private var threshold_internal: Double = fittingParamMap.get(threshold).getOrElse(getThreshold) - private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept } @@ -142,7 +133,8 @@ class LogisticRegressionModel private[ml] ( val scoreFunction = udf { v: Vector => val margin = BLAS.dot(v, weights) 1.0 / (1.0 + math.exp(-margin)) - val t = threshold_internal + } + val t = map(threshold) val predictFunction: Double => Double = (score) => { if (score > t) 1.0 else 0.0 } dataset .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) @@ -151,12 +143,14 @@ class LogisticRegressionModel private[ml] ( override val numClasses: Int = 2 + // TODO: Override batch predict() for efficiency. + /** * Predict label for the given feature vector. * The behavior of this can be adjusted using [[threshold]]. */ override def predict(features: Vector): Double = { - if (score(features) > threshold_internal) 1 else 0 + if (score(features) > paramMap(threshold)) 1 else 0 } override def predictProbabilities(features: Vector): Vector = { @@ -168,4 +162,10 @@ class LogisticRegressionModel private[ml] ( val m = margin(features) Vectors.dense(Array(-m, m)) } + + private[ml] override def copy(): LogisticRegressionModel = { + val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) + Params.inheritValues(this.paramMap, this, m) + m + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 709ed3a696e49..d7d10c59b751e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -128,22 +128,33 @@ private[ml] abstract class PredictionModel[M <: PredictionModel[M]] transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) val pred: Vector => Double = (features) => { - predict(features) + tmpModel.predict(features) } dataset.select(Star(None), pred.call(map(featuresCol).attr) as map(predictionCol)) } /** - * Default implementation. - * Override for efficiency; e.g., this does not broadcast the model. + * Default implementation using single-instance predict(). + * + * Developers should override this for efficiency. E.g., this does not broadcast the model. */ - def predict(dataset: RDD[Vector]): RDD[Double] = { - dataset.map(predict) + def predict(dataset: RDD[Vector], paramMap: ParamMap): RDD[Double] = { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + dataset.map(tmpModel.predict) } /** * Predict label for the given features. */ def predict(features: Vector): Double + + /** + * Create a copy of the model. + * The copy is shallow, except for the embedded paramMap, which gets a deep copy. + */ + private[ml] def copy(): M } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index c34746e9187da..57817af02ddbd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.LabeledPoint -import org.apache.spark.ml.param.{ParamMap, HasMaxIter, HasRegParam} +import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.rdd.RDD @@ -89,4 +89,10 @@ class LinearRegressionModel private[ml] ( override def predict(features: Vector): Double = { BLAS.dot(features, weights) + intercept } + + private[ml] override def copy(): LinearRegressionModel = { + val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) + Params.inheritValues(this.paramMap, this, m) + m + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 33e40dc7410cc..78f45576622bb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -21,7 +21,8 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{SQLContext, DataFrame} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { @@ -32,21 +33,29 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { super.beforeAll() sqlContext = new SQLContext(sc) dataset = sqlContext.createDataFrame( - sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2)) + sc.parallelize(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42), 2)) } - test("logistic regression") { + test("logistic regression: default params") { val lr = new LogisticRegression val model = lr.fit(dataset) model.transform(dataset) .select("label", "prediction") .collect() + // Check defaults + assert(model.getThreshold === 0.5) + assert(model.getFeaturesCol == "features") + assert(model.getPredictionCol == "prediction") + assert(model.getScoreCol == "score") } test("logistic regression with setters") { + // Set params, train, and check as many as we can. val lr = new LogisticRegression() .setMaxIter(10) .setRegParam(1.0) + .setThreshold(0.6) + .setScoreCol("probability") val model = lr.fit(dataset) model.transform(dataset, model.threshold -> 0.8) // overwrite threshold .select("label", "score", "prediction") @@ -58,6 +67,33 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { val model = lr.fit(dataset, lr.maxIter -> 10, lr.regParam -> 1.0) model.transform(dataset, model.threshold -> 0.8, model.scoreCol -> "probability") .select("label", "probability", "prediction") + assert(model.fittingParamMap.get(lr.maxIter) === Some(10)) + assert(model.fittingParamMap.get(lr.regParam) === Some(1.0)) + assert(model.fittingParamMap.get(lr.threshold) === Some(0.6)) + assert(model.getThreshold === 0.6) + + // Modify model params, and check that they work. + model.setThreshold(1.0) + val predAllZero = model.transform(dataset) + .select('prediction, 'probability) .collect() + .map { case Row(pred: Double, prob: Double) => pred } + assert(predAllZero.forall(_ === 0.0)) + // Call transform with params, and check that they work. + val predNotAllZero = + model.transform(dataset, model.threshold -> 0.0, model.scoreCol -> "myProb") + .select('prediction, 'myProb) + .collect() + .map { case Row(pred: Double, prob: Double) => pred } + assert(predNotAllZero.exists(_ !== 0.0)) + + // Call fit() with new params, and check as many as we can. + val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.threshold -> 0.4, + lr.scoreCol -> "theProb") + assert(model2.fittingParamMap.get(lr.maxIter) === Some(5)) + assert(model2.fittingParamMap.get(lr.regParam) === Some(0.1)) + assert(model2.fittingParamMap.get(lr.threshold) === Some(0.4)) + assert(model2.getThreshold === 0.4) + assert(model2.getScoreCol == "theProb") } } From 58802e3dccde601c47c16f462a0880a2cda58a81 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 5 Dec 2014 17:32:14 -0800 Subject: [PATCH 10/30] added train() to Predictor subclasses which does not take a ParamMap. --- .../ml/classification/LogisticRegression.scala | 13 ++++++++++--- .../spark/ml/regression/LinearRegression.scala | 13 ++++++++++--- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index cfb3a6bda041f..a2d8a1572b981 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -70,12 +70,12 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi /** * Same as [[fit()]], but using strong types. - * - * @param dataset Training data. WARNING: This does not yet handle instance weights. + * NOTE: This does NOT support instance weights. + * @param dataset Training data. Instance weights are ignored. * @param paramMap Parameters for training. * These values override any specified in this Estimator's embedded ParamMap. */ - def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LogisticRegressionModel = { + override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LogisticRegressionModel = { val map = this.paramMap ++ paramMap val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) @@ -96,6 +96,13 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi } lrm } + + /** + * Same as [[fit()]], but using strong types. + * NOTE: This does NOT support instance weights. + * @param dataset Training data. Instance weights are ignored. + */ + def train(dataset: RDD[LabeledPoint]): LogisticRegressionModel = train(dataset, new ParamMap()) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 57817af02ddbd..10308ac9a8a33 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -47,12 +47,12 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel /** * Same as [[fit()]], but using strong types. - * - * @param dataset Training data. WARNING: This does not yet handle instance weights. + * NOTE: This does NOT support instance weights. + * @param dataset Training data. Instance weights are ignored. * @param paramMap Parameters for training. * These values override any specified in this Estimator's embedded ParamMap. */ - def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { + override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) } @@ -71,6 +71,13 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel } lrm } + + /** + * Same as [[fit()]], but using strong types. + * NOTE: This does NOT support instance weights. + * @param dataset Training data. Instance weights are ignored. + */ + def train(dataset: RDD[LabeledPoint]): LinearRegressionModel = train(dataset, new ParamMap()) } /** From adbe50a7bfc4af7e6771dcd1d6a5f370a9490eca Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 5 Dec 2014 19:57:12 -0800 Subject: [PATCH 11/30] * fixed LinearRegression train() to use embedded paramMap * added Predictor.predict(RDD[Vector]) method * updated Linear/LogisticRegressionSuites --- .../spark/ml/impl/estimator/Predictor.scala | 4 + .../ml/regression/LinearRegression.scala | 7 +- .../LogisticRegressionSuite.scala | 44 +++++++++ .../ml/regression/LinearRegressionSuite.scala | 95 +++++++++++++++++++ 4 files changed, 147 insertions(+), 3 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index d7d10c59b751e..c8dc5736f9cd1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -137,6 +137,7 @@ private[ml] abstract class PredictionModel[M <: PredictionModel[M]] } /** + * Strongly typed version of [[transform()]]. * Default implementation using single-instance predict(). * * Developers should override this for efficiency. E.g., this does not broadcast the model. @@ -147,6 +148,9 @@ private[ml] abstract class PredictionModel[M <: PredictionModel[M]] dataset.map(tmpModel.predict) } + /** Strongly typed version of [[transform()]]. */ + def predict(dataset: RDD[Vector]): RDD[Double] = predict(dataset, new ParamMap) + /** * Predict label for the given features. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 10308ac9a8a33..2c7f4a7efcf1c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -53,6 +53,7 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel * These values override any specified in this Estimator's embedded ParamMap. */ override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { + val map = this.paramMap ++ paramMap val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => org.apache.spark.mllib.regression.LabeledPoint(label, features) } @@ -62,10 +63,10 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel } val lr = new LinearRegressionWithSGD() lr.optimizer - .setRegParam(paramMap(regParam)) - .setNumIterations(paramMap(maxIter)) + .setRegParam(map(regParam)) + .setNumIterations(map(maxIter)) val model = lr.run(oldDataset) - val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) + val lrm = new LinearRegressionModel(this, map, model.weights, model.intercept) if (handlePersistence) { oldDataset.unpersist() } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 78f45576622bb..b2aa3a1573ea1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -19,8 +19,11 @@ package org.apache.spark.ml.classification import org.scalatest.FunSuite +import org.apache.spark.ml.LabeledPoint import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} @@ -28,6 +31,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { @transient var sqlContext: SQLContext = _ @transient var dataset: DataFrame = _ + private val eps: Double = 1e-5 override def beforeAll(): Unit = { super.beforeAll() @@ -38,6 +42,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { test("logistic regression: default params") { val lr = new LogisticRegression + assert(lr.getLabelCol == "label") val model = lr.fit(dataset) model.transform(dataset) .select("label", "prediction") @@ -96,4 +101,43 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(model2.getThreshold === 0.4) assert(model2.getScoreCol == "theProb") } + + test("logistic regression: Predictor, Classifier methods") { + val sqlContext = this.sqlContext + import sqlContext._ + val lr = new LogisticRegression + + // fit() vs. train() + val model1 = lr.fit(dataset) + val rdd = dataset.select('label, 'features).map { case Row(label: Double, features: Vector) => + LabeledPoint(label, features) + } + val features = rdd.map(_.features) + val model2 = lr.train(rdd) + assert(model1.intercept == model2.intercept) + assert(model1.weights.equals(model2.weights)) + assert(model1.numClasses == model2.numClasses) + assert(model1.numClasses === 2) + + // transform() vs. predict() + val trans = model1.transform(dataset).select('prediction) + val preds = model1.predict(rdd.map(_.features)) + trans.zip(preds).collect().foreach { case (Row(pred1: Double), pred2: Double) => + assert(pred1 == pred2) + } + + // Check various types of predictions. + val allPredictions = features.map { f => + (model1.predictRaw(f), model1.predictProbabilities(f), model1.predict(f)) + }.collect() + val threshold = model1.getThreshold + allPredictions.foreach { case (raw: Vector, prob: Vector, pred: Double) => + val computeProbFromRaw: (Double => Double) = (m) => 1.0 / (1.0 + math.exp(-m)) + raw.toArray.map(computeProbFromRaw).zip(prob.toArray).foreach { case (r, p) => + assert(r ~== p relTol eps) + } + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala new file mode 100644 index 0000000000000..ec24d1eae08c1 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.regression + +import org.scalatest.FunSuite + +import org.apache.spark.ml.LabeledPoint +import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{Row, SQLContext, SchemaRDD} + +class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + @transient var dataset: SchemaRDD = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + dataset = sqlContext.createSchemaRDD( + sc.parallelize(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42), 2)) + } + + test("linear regression: default params") { + val sqlContext = this.sqlContext + import sqlContext._ + val lr = new LinearRegression + assert(lr.getLabelCol == "label") + val model = lr.fit(dataset) + model.transform(dataset) + .select('label, 'prediction) + .collect() + // Check defaults + assert(model.getFeaturesCol == "features") + assert(model.getPredictionCol == "prediction") + } + + test("linear regression with setters") { + // Set params, train, and check as many as we can. + val sqlContext = this.sqlContext + import sqlContext._ + val lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(1.0) + val model = lr.fit(dataset) + assert(model.fittingParamMap.get(lr.maxIter) === Some(10)) + assert(model.fittingParamMap.get(lr.regParam) === Some(1.0)) + + // Call fit() with new params, and check as many as we can. + val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.predictionCol -> "thePred") + assert(model2.fittingParamMap.get(lr.maxIter) === Some(5)) + assert(model2.fittingParamMap.get(lr.regParam) === Some(0.1)) + assert(model2.getPredictionCol == "thePred") + } + + test("linear regression: Predictor, Regressor methods") { + val sqlContext = this.sqlContext + import sqlContext._ + val lr = new LinearRegression + + // fit() vs. train() + val model1 = lr.fit(dataset) + val rdd = dataset.select('label, 'features).map { case Row(label: Double, features: Vector) => + LabeledPoint(label, features) + } + val features = rdd.map(_.features) + val model2 = lr.train(rdd) + assert(model1.intercept == model2.intercept) + assert(model1.weights.equals(model2.weights)) + + // transform() vs. predict() + val trans = model1.transform(dataset).select('prediction) + val preds = model1.predict(rdd.map(_.features)) + trans.zip(preds).collect().foreach { case (Row(pred1: Double), pred2: Double) => + assert(pred1 == pred2) + } + } +} From 16809055f4cc38dfee85b150d53cb8805a3e4ad1 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 5 Dec 2014 22:18:30 -0800 Subject: [PATCH 12/30] Added JavaLabeledPointSuite.java for spark.ml, and added constructor to LabeledPoint which defaults weight to 1.0 --- .../org/apache/spark/ml/LabeledPoint.scala | 3 + .../spark/ml/JavaLabeledPointSuite.java | 61 +++++++++++++++++++ .../apache/spark/ml/LabeledPointSuite.scala | 6 +- 3 files changed, 68 insertions(+), 2 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala index 8a310fc7b1fee..8b6b2f3fa2756 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala @@ -34,6 +34,9 @@ import org.apache.spark.mllib.linalg.Vector @BeanInfo case class LabeledPoint(label: Double, features: Vector, weight: Double) { + /** Constructor which sets instance weight to 1.0 */ + def this(label: Double, features: Vector) = this(label, features, 1.0) + override def toString: String = { "(%s,%s,%s)".format(label, features, weight) } diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java new file mode 100644 index 0000000000000..ac6cb7aa3b344 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java @@ -0,0 +1,61 @@ +package org.apache.spark.ml; + +import java.util.List; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +/** + * Test {@link LabeledPoint} in Java + */ +public class JavaLabeledPointSuite { + + private transient JavaSparkContext jsc; + private transient JavaSQLContext jsql; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaLabeledPointSuite"); + jsql = new JavaSQLContext(jsc); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + } + + @Test + public void labeledPointDefaultWeight() { + double label = 1.0; + Vector features = Vectors.dense(1.0, 2.0, 3.0); + LabeledPoint lp1 = new LabeledPoint(label, features); + LabeledPoint lp2 = new LabeledPoint(label, features, 1.0); + assert(lp1.equals(lp2)); + } + + @Test + public void labeledPointSchemaRDD() { + List arr = Lists.newArrayList( + new LabeledPoint(0.0, Vectors.dense(1.0, 2.0, 3.0)), + new LabeledPoint(1.0, Vectors.dense(1.1, 2.1, 3.1)), + new LabeledPoint(0.0, Vectors.dense(1.2, 2.2, 3.2)), + new LabeledPoint(1.0, Vectors.dense(1.3, 2.3, 3.3))); + JavaRDD rdd = jsc.parallelize(arr); + JavaSchemaRDD schemaRDD = jsql.applySchema(rdd, LabeledPoint.class); + schemaRDD.registerTempTable("points"); + List points = jsql.sql("SELECT label, features FROM points").collect(); + assert (points.size() == arr.size()); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala index 34460a9e21d0d..94659ba95b1be 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala @@ -19,11 +19,13 @@ package org.apache.spark.ml import org.scalatest.FunSuite -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{SQLContext, SchemaRDD} +import org.apache.spark.sql.SQLContext +/** + * Test [[LabeledPoint]] + */ class LabeledPointSuite extends FunSuite with MLlibTestSparkContext { @transient var sqlContext: SQLContext = _ From 8d13233150997e9564eec30fde4b886b675f240c Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 8 Dec 2014 11:09:03 -0800 Subject: [PATCH 13/30] Added methods: * Classifier: batch predictRaw() * Predictor: train() without paramMap ProbabilisticClassificationModel.predictProbabilities() * Java versions of all above batch methods + others Updated LogisticRegressionSuite. Updated JavaLogisticRegressionSuite to match LogisticRegressionSuite. --- .../spark/ml/classification/Classifier.scala | 10 ++ .../classification/LogisticRegression.scala | 3 +- .../spark/ml/impl/estimator/Predictor.scala | 23 ++++ .../ProbabilisticClassificationModel.scala | 9 ++ .../ml/regression/LinearRegression.scala | 3 +- .../JavaLogisticRegressionSuite.java | 120 +++++++++++++++++- .../LogisticRegressionSuite.scala | 28 ++-- 7 files changed, 174 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index eeef116c876db..2f31beb7303fb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -18,8 +18,10 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD /** * Params for classification. @@ -72,6 +74,14 @@ abstract class ClassificationModel[M <: ClassificationModel[M]] */ def predictRaw(features: Vector): Vector + /** Batch version of [[predictRaw]] */ + def predictRaw(dataset: RDD[Vector]): RDD[Vector] = dataset.map(predictRaw) + + /** Java-friendly batch version of [[predictRaw]] */ + def predictRaw(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { + dataset.rdd.map(predictRaw).toJavaRDD() + } + // TODO: accuracy(dataset: RDD[LabeledPoint]): Double (follow-up PR) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index a2d8a1572b981..b8628d6cb8778 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -102,7 +102,8 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi * NOTE: This does NOT support instance weights. * @param dataset Training data. Instance weights are ignored. */ - def train(dataset: RDD[LabeledPoint]): LogisticRegressionModel = train(dataset, new ParamMap()) + override def train(dataset: RDD[LabeledPoint]): LogisticRegressionModel = + train(dataset, new ParamMap()) // Override documentation } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index c8dc5736f9cd1..48cecfefd4c07 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.impl.estimator +import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.{Estimator, LabeledPoint, Model} import org.apache.spark.ml.param._ import org.apache.spark.mllib.linalg.{Vector, VectorUDT} @@ -101,6 +102,18 @@ private[ml] abstract class Predictor[Learner <: Predictor[Learner, M], M <: Pred * These values override any specified in this Estimator's embedded ParamMap. */ def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): M + + /** + * Same as [[fit()]], but using strong types. + * @param dataset Training data + */ + def train(dataset: RDD[LabeledPoint]): M = train(dataset, new ParamMap()) + + /** Java-friendly version of [[train()]]. */ + def train(dataset: JavaRDD[LabeledPoint], paramMap: ParamMap): M = train(dataset.rdd, paramMap) + + /** Java-friendly version of [[train()]]. */ + def train(dataset: JavaRDD[LabeledPoint]): M = train(dataset.rdd) } private[ml] abstract class PredictionModel[M <: PredictionModel[M]] @@ -156,6 +169,16 @@ private[ml] abstract class PredictionModel[M <: PredictionModel[M]] */ def predict(features: Vector): Double + /** Java-friendly version of [[predict()]]. */ + def predict(dataset: JavaRDD[Vector], paramMap: ParamMap): JavaRDD[java.lang.Double] = { + predict(dataset.rdd, paramMap).map(_.asInstanceOf[java.lang.Double]).toJavaRDD() + } + + /** Java-friendly version of [[predict()]]. */ + def predict(dataset: JavaRDD[Vector]): JavaRDD[java.lang.Double] = { + predict(dataset.rdd, new ParamMap).map(_.asInstanceOf[java.lang.Double]).toJavaRDD() + } + /** * Create a copy of the model. * The copy is shallow, except for the embedded paramMap, which gets a deep copy. diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala index 52096ce6b2589..e534a8c264bb3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala @@ -17,7 +17,9 @@ package org.apache.spark.ml.impl.estimator +import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD /** * Trait for a [[org.apache.spark.ml.classification.ClassificationModel]] which can output @@ -34,4 +36,11 @@ private[ml] trait ProbabilisticClassificationModel { */ def predictProbabilities(features: Vector): Vector + /** Batch version of [[predictProbabilities()]] */ + def predictProbabilities(features: RDD[Vector]): RDD[Vector] = features.map(predictProbabilities) + + /** Java-friendly batch version of [[predictProbabilities()]] */ + def predictProbabilities(features: JavaRDD[Vector]): JavaRDD[Vector] = { + features.rdd.map(predictProbabilities).toJavaRDD() + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 2c7f4a7efcf1c..e6abe9b404808 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -78,7 +78,8 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel * NOTE: This does NOT support instance weights. * @param dataset Training data. Instance weights are ignored. */ - def train(dataset: RDD[LabeledPoint]): LinearRegressionModel = train(dataset, new ParamMap()) + override def train(dataset: RDD[LabeledPoint]): LinearRegressionModel = + train(dataset, new ParamMap()) // Override documentation } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index f4ba23c44563e..3ad15e516c16e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -17,18 +17,28 @@ package org.apache.spark.ml.classification; +import scala.Tuple2; + import java.io.Serializable; +import java.lang.Math; +import java.util.ArrayList; import java.util.List; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.SQLContext; import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.LabeledPoint; +import org.apache.spark.sql.Row; + public class JavaLogisticRegressionSuite implements Serializable { @@ -36,12 +46,25 @@ public class JavaLogisticRegressionSuite implements Serializable { private transient SQLContext jsql; private transient DataFrame dataset; + private transient JavaRDD datasetRDD; + private transient JavaRDD featuresRDD; + private double eps = 1e-5; + @Before public void setUp() { jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); jsql = new SQLContext(jsc); - List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); - dataset = jsql.applySchema(jsc.parallelize(points, 2), LabeledPoint.class); + List points = new ArrayList(); + for (org.apache.spark.mllib.regression.LabeledPoint lp: + generateLogisticInputAsList(1.0, 1.0, 100, 42)) { + points.add(new LabeledPoint(lp.label(), lp.features())); + } + datasetRDD = jsc.parallelize(points, 2); + featuresRDD = datasetRDD.map(new Function() { + @Override public Vector call(LabeledPoint lp) { return lp.features(); } + }); + dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); + dataset.registerTempTable("dataset"); } @After @@ -51,29 +74,112 @@ public void tearDown() { } @Test - public void logisticRegression() { + public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); + assert(lr.getLabelCol().equals("label")); LogisticRegressionModel model = lr.fit(dataset); model.transform(dataset).registerTempTable("prediction"); DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); predictions.collectAsList(); + // Check defaults + assert(model.getThreshold() == 0.5); + assert(model.getFeaturesCol().equals("features")); + assert(model.getPredictionCol().equals("prediction")); + assert(model.getScoreCol().equals("score")); } @Test public void logisticRegressionWithSetters() { + // Set params, train, and check as many params as we can. LogisticRegression lr = new LogisticRegression() .setMaxIter(10) - .setRegParam(1.0); + .setRegParam(1.0) + .setThreshold(0.6) + .setScoreCol("probability"); LogisticRegressionModel model = lr.fit(dataset); + assert(model.fittingParamMap().get(lr.maxIter()).get() == 10); + assert(model.fittingParamMap().get(lr.regParam()).get() == 1.0); + assert(model.fittingParamMap().get(lr.threshold()).get() == 0.6); + assert(model.getThreshold() == 0.6); + + // Modify model params, and check that the params worked. + model.setThreshold(1.0); + model.transform(dataset).registerTempTable("predAllZero"); + SchemaRDD predAllZero = jsql.sql("SELECT prediction, probability FROM predAllZero"); + for (Row r: predAllZero.collectAsList()) { + assert(r.getDouble(0) == 0.0); + } + // Call transform with params, and check that the params worked. + /* TODO: USE THIS model.transform(dataset, model.threshold().w(0.8)) // overwrite threshold - .registerTempTable("prediction"); + .registerTempTable("prediction"); DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); predictions.collectAsList(); + */ + + model.transform(dataset, model.threshold().w(0.0), model.scoreCol().w("myProb")) + .registerTempTable("predNotAllZero"); + SchemaRDD predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); + boolean foundNonZero = false; + for (Row r: predNotAllZero.collectAsList()) { + if (r.getDouble(0) != 0.0) foundNonZero = true; + } + assert(foundNonZero); + + // Call fit() with new params, and check as many params as we can. + LogisticRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), + lr.threshold().w(0.4), lr.scoreCol().w("theProb")); + assert(model2.fittingParamMap().get(lr.maxIter()).get() == 5); + assert(model2.fittingParamMap().get(lr.regParam()).get() == 0.1); + assert(model2.fittingParamMap().get(lr.threshold()).get() == 0.4); + assert(model2.getThreshold() == 0.4); + assert(model2.getScoreCol().equals("theProb")); } @Test - public void logisticRegressionFitWithVarargs() { + public void logisticRegressionPredictorClassifierMethods() { LogisticRegression lr = new LogisticRegression(); - lr.fit(dataset, lr.maxIter().w(10), lr.regParam().w(1.0)); + + // fit() vs. train() + LogisticRegressionModel model1 = lr.fit(dataset); + LogisticRegressionModel model2 = lr.train(datasetRDD); + assert(model1.intercept() == model2.intercept()); + assert(model1.weights().equals(model2.weights())); + assert(model1.numClasses() == model2.numClasses()); + assert(model1.numClasses() == 2); + + // transform() vs. predict() + model1.transform(dataset).registerTempTable("transformed"); + SchemaRDD trans = jsql.sql("SELECT prediction FROM transformed"); + JavaRDD preds = model1.predict(featuresRDD); + for (scala.Tuple2 trans_pred: trans.toJavaRDD().zip(preds).collect()) { + double t = trans_pred._1().getDouble(0); + double p = trans_pred._2(); + assert(t == p); + } + + // Check various types of predictions. + JavaRDD rawPredictions = model1.predictRaw(featuresRDD); + JavaRDD probabilities = model1.predictProbabilities(featuresRDD); + JavaRDD predictions = model1.predict(featuresRDD); + double threshold = model1.getThreshold(); + for (Tuple2 raw_prob: rawPredictions.zip(probabilities).collect()) { + Vector raw = raw_prob._1(); + Vector prob = raw_prob._2(); + for (int i = 0; i < raw.size(); ++i) { + double r = raw.apply(i); + double p = prob.apply(i); + double pFromR = 1.0 / (1.0 + Math.exp(-r)); + assert(Math.abs(r - pFromR) < eps); + } + } + for (Tuple2 prob_pred: probabilities.zip(predictions).collect()) { + Vector prob = prob_pred._1(); + double pred = prob_pred._2(); + double probOfPred = prob.apply((int)pred); + for (int i = 0; i < prob.size(); ++i) { + assert(probOfPred >= prob.apply(i)); + } + } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index b2aa3a1573ea1..df2374bf655e9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -45,7 +45,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(lr.getLabelCol == "label") val model = lr.fit(dataset) model.transform(dataset) - .select("label", "prediction") + .select('label, 'score, 'prediction) .collect() // Check defaults assert(model.getThreshold === 0.5) @@ -55,7 +55,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { } test("logistic regression with setters") { - // Set params, train, and check as many as we can. + // Set params, train, and check as many params as we can. val lr = new LogisticRegression() .setMaxIter(10) .setRegParam(1.0) @@ -77,14 +77,14 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(model.fittingParamMap.get(lr.threshold) === Some(0.6)) assert(model.getThreshold === 0.6) - // Modify model params, and check that they work. + // Modify model params, and check that the params worked. model.setThreshold(1.0) val predAllZero = model.transform(dataset) .select('prediction, 'probability) .collect() .map { case Row(pred: Double, prob: Double) => pred } assert(predAllZero.forall(_ === 0.0)) - // Call transform with params, and check that they work. + // Call transform with params, and check that the params worked. val predNotAllZero = model.transform(dataset, model.threshold -> 0.0, model.scoreCol -> "myProb") .select('prediction, 'myProb) @@ -92,12 +92,12 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { .map { case Row(pred: Double, prob: Double) => pred } assert(predNotAllZero.exists(_ !== 0.0)) - // Call fit() with new params, and check as many as we can. + // Call fit() with new params, and check as many params as we can. val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.threshold -> 0.4, lr.scoreCol -> "theProb") - assert(model2.fittingParamMap.get(lr.maxIter) === Some(5)) - assert(model2.fittingParamMap.get(lr.regParam) === Some(0.1)) - assert(model2.fittingParamMap.get(lr.threshold) === Some(0.4)) + assert(model2.fittingParamMap.get(lr.maxIter).get === 5) + assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) + assert(model2.fittingParamMap.get(lr.threshold).get === 0.4) assert(model2.getThreshold === 0.4) assert(model2.getScoreCol == "theProb") } @@ -112,7 +112,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { val rdd = dataset.select('label, 'features).map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } - val features = rdd.map(_.features) + val featuresRDD = rdd.map(_.features) val model2 = lr.train(rdd) assert(model1.intercept == model2.intercept) assert(model1.weights.equals(model2.weights)) @@ -127,15 +127,17 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { } // Check various types of predictions. - val allPredictions = features.map { f => - (model1.predictRaw(f), model1.predictProbabilities(f), model1.predict(f)) - }.collect() + val rawPredictions = model1.predictRaw(featuresRDD) + val probabilities = model1.predictProbabilities(featuresRDD) + val predictions = model1.predict(featuresRDD) val threshold = model1.getThreshold - allPredictions.foreach { case (raw: Vector, prob: Vector, pred: Double) => + rawPredictions.zip(probabilities).collect().foreach { case (raw: Vector, prob: Vector) => val computeProbFromRaw: (Double => Double) = (m) => 1.0 / (1.0 + math.exp(-m)) raw.toArray.map(computeProbFromRaw).zip(prob.toArray).foreach { case (r, p) => assert(r ~== p relTol eps) } + } + probabilities.zip(predictions).collect().foreach { case (prob: Vector, pred: Double) => val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 assert(pred == predFromProb) } From bc654e15a55312725345156ad4ad2ef6a3c31335 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 8 Dec 2014 11:51:55 -0800 Subject: [PATCH 14/30] Added spark.ml LinearRegressionSuite --- .../JavaLinearRegressionSuite.java | 127 ++++++++++++++++++ .../ml/regression/LinearRegressionSuite.scala | 8 +- 2 files changed, 131 insertions(+), 4 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java new file mode 100644 index 0000000000000..8662d68cd365b --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java @@ -0,0 +1,127 @@ +/* + * 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 scala.Tuple2; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.ml.LabeledPoint; +import org.apache.spark.ml.regression.LinearRegression; +import org.apache.spark.ml.regression.LinearRegressionModel; +import static org.apache.spark.mllib.classification.LogisticRegressionSuite + .generateLogisticInputAsList; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + + +public class JavaLinearRegressionSuite implements Serializable { + + private transient JavaSparkContext jsc; + private transient JavaSQLContext jsql; + private transient JavaSchemaRDD dataset; + private transient JavaRDD datasetRDD; + private transient JavaRDD featuresRDD; + private double eps = 1e-5; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); + jsql = new JavaSQLContext(jsc); + List points = new ArrayList(); + for (org.apache.spark.mllib.regression.LabeledPoint lp: + generateLogisticInputAsList(1.0, 1.0, 100, 42)) { + points.add(new LabeledPoint(lp.label(), lp.features())); + } + datasetRDD = jsc.parallelize(points, 2); + featuresRDD = datasetRDD.map(new Function() { + @Override public Vector call(LabeledPoint lp) { return lp.features(); } + }); + dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); + dataset.registerTempTable("dataset"); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + } + + @Test + public void linearRegressionDefaultParams() { + LinearRegression lr = new LinearRegression(); + assert(lr.getLabelCol().equals("label")); + LinearRegressionModel model = lr.fit(dataset); + model.transform(dataset).registerTempTable("prediction"); + JavaSchemaRDD predictions = jsql.sql("SELECT label, prediction FROM prediction"); + predictions.collect(); + // Check defaults + assert(model.getFeaturesCol().equals("features")); + assert(model.getPredictionCol().equals("prediction")); + } + + @Test + public void linearRegressionWithSetters() { + // Set params, train, and check as many params as we can. + LinearRegression lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(1.0); + LinearRegressionModel model = lr.fit(dataset); + assert(model.fittingParamMap().get(lr.maxIter()).get() == 10); + assert(model.fittingParamMap().get(lr.regParam()).get() == 1.0); + + // Call fit() with new params, and check as many params as we can. + LinearRegressionModel model2 = + lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); + assert(model2.fittingParamMap().get(lr.maxIter()).get() == 5); + assert(model2.fittingParamMap().get(lr.regParam()).get() == 0.1); + assert(model2.getPredictionCol().equals("thePred")); + } + + @Test + public void linearRegressionPredictorClassifierMethods() { + LinearRegression lr = new LinearRegression(); + + // fit() vs. train() + LinearRegressionModel model1 = lr.fit(dataset); + LinearRegressionModel model2 = lr.train(datasetRDD); + assert(model1.intercept() == model2.intercept()); + assert(model1.weights().equals(model2.weights())); + + // transform() vs. predict() + model1.transform(dataset).registerTempTable("transformed"); + JavaSchemaRDD trans = jsql.sql("SELECT prediction FROM transformed"); + JavaRDD preds = model1.predict(featuresRDD); + for (Tuple2 trans_pred: trans.zip(preds).collect()) { + double t = trans_pred._1().getDouble(0); + double p = trans_pred._2(); + assert(t == p); + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index ec24d1eae08c1..b126ccabfb37e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -60,13 +60,13 @@ class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { .setMaxIter(10) .setRegParam(1.0) val model = lr.fit(dataset) - assert(model.fittingParamMap.get(lr.maxIter) === Some(10)) - assert(model.fittingParamMap.get(lr.regParam) === Some(1.0)) + assert(model.fittingParamMap.get(lr.maxIter).get === 10) + assert(model.fittingParamMap.get(lr.regParam).get === 1.0) // Call fit() with new params, and check as many as we can. val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.predictionCol -> "thePred") - assert(model2.fittingParamMap.get(lr.maxIter) === Some(5)) - assert(model2.fittingParamMap.get(lr.regParam) === Some(0.1)) + assert(model2.fittingParamMap.get(lr.maxIter).get === 5) + assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) assert(model2.getPredictionCol == "thePred") } From 4e2f711ff0cd3faf59f578ce758209c93924aedc Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 8 Dec 2014 12:15:00 -0800 Subject: [PATCH 15/30] rat fix --- .../apache/spark/ml/JavaLabeledPointSuite.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java index ac6cb7aa3b344..878ad5a7baed5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java @@ -1,3 +1,20 @@ +/* + * 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; import java.util.List; From 1c617231a739f3ecea6bf21ba6ab5a49cb0a7c29 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 30 Dec 2014 13:41:26 -0800 Subject: [PATCH 16/30] =?UTF-8?q?*=20Made=20ProbabilisticClassificationMod?= =?UTF-8?q?el=20into=20a=20subclass=20of=20ClassificationModel.=20=20Also?= =?UTF-8?q?=20introduced=20ProbabilisticClassifier.=20=20*=20This=20was=20?= =?UTF-8?q?to=20support=20output=20column=20=E2=80=9CprobabilityCol?= =?UTF-8?q?=E2=80=9D=20in=20transform().?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * SPARK-4942 : ML Transformers should allow output cols to be turned on,off * Update validateAndTransformSchema * Update transform * Update based on design review * Make prediction API protected, but add output columns * Remove training API * LogisticRegression: * Changed output column “score” to “probability” in logreg. * I also implemented transform() to avoid repeated computation. This improves upon the default implementation in ProbabilisticClassificationModel. However, it’s a lot of code, so I would be fine with removing it. There is also a question of whether all algorithms should implement a method which would allow the ProbabilisticClassificationModel.transform implementation to avoid repeated computation: * protected def raw2prob(rawPredictions: Vector): Vector = // compute probabilities from raw predictions * trait Params: * Changed set() and get() from private[ml] to protected. This was needed for the example of defining a class from outside of the MLlib namespace. * VectorUDT: Changed from private[spark] to public. This is needed for outside users to write their own validateAndTransformSchema() methods using vectors. * Add example of defining class from outside of the MLlib namespace. * Scala --- .../ml/JavaCrossValidatorExample.java | 4 +- .../examples/ml/JavaSimpleParamsExample.java | 6 +- .../JavaSimpleTextClassificationPipeline.java | 2 +- .../examples/ml/CrossValidatorExample.scala | 7 +- .../examples/ml/DeveloperApiExample.scala | 197 ++++++++++++++++++ .../examples/ml/SimpleParamsExample.scala | 14 +- .../ml/SimpleTextClassificationPipeline.scala | 7 +- .../org/apache/spark/ml/LabeledPoint.scala | 52 ----- .../spark/ml/classification/Classifier.scala | 161 ++++++++++++-- .../classification/LogisticRegression.scala | 151 +++++++++----- .../ProbabilisticClassifier.scala | 145 +++++++++++++ .../BinaryClassificationEvaluator.scala | 20 +- .../apache/spark/ml/feature/Tokenizer.scala | 4 +- .../spark/ml/impl/estimator/Predictor.scala | 182 ++++++++-------- .../ProbabilisticClassificationModel.scala | 46 ---- .../org/apache/spark/ml/param/params.scala | 45 +++- .../apache/spark/ml/param/sharedParams.scala | 26 ++- .../ml/regression/LinearRegression.scala | 48 ++--- .../spark/ml/regression/Regressor.scala | 32 ++- .../apache/spark/mllib/linalg/Vectors.scala | 9 +- 20 files changed, 820 insertions(+), 338 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index 0fbee6e433608..7fc44739b6ca7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -116,9 +116,9 @@ public static void main(String[] args) { // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT id, text, probability, prediction FROM prediction"); for (Row r: predictions.collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index eaaa344be49c8..3f4f42594059a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -81,7 +81,7 @@ public static void main(String[] args) { // One can also combine ParamMaps. ParamMap paramMap2 = new ParamMap(); - paramMap2.put(lr.scoreCol().w("probability")); // Change output column name + paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); // Now learn a new model using the paramMapCombined parameters. @@ -98,8 +98,8 @@ public static void main(String[] args) { // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. - // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' - // column since we renamed the lr.scoreCol parameter previously. + // Note that model2.transform() outputs a 'myProbability' column instead of the usual + // 'probability' column since we renamed the lr.probabilityCol parameter previously. model2.transform(test).registerTempTable("results"); DataFrame results = jsql.sql("SELECT features, label, probability, prediction FROM results"); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index 82d665a3e1386..f27550e7337dd 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -85,7 +85,7 @@ public static void main(String[] args) { model.transform(test).registerTempTable("prediction"); DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); for (Row r: predictions.collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index b6c30a007d88f..20fcf132be15b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -23,6 +23,7 @@ import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{Row, SQLContext} /** @@ -100,10 +101,10 @@ object CrossValidatorExample { // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test) - .select("id", "text", "score", "prediction") + .select('id, 'text, 'probability, 'prediction) .collect() - .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => - println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => + println("(" + id + ", " + text + ") --> prob=" + prob + ", prediction=" + prediction) } sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala new file mode 100644 index 0000000000000..2f1de5c58ed1e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -0,0 +1,197 @@ +/* + * 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.examples.ml + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.classification.{Classifier, ClassifierParams, ClassificationModel} +import org.apache.spark.ml.param.{Params, IntParam, ParamMap} +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.sql.{DataType, SchemaRDD, Row, SQLContext} + +/** + * A simple example demonstrating how to write your own learning algorithm using Estimator, + * Transformer, and other abstractions. + * This mimics [[org.apache.spark.ml.classification.LogisticRegression]]. + * Run with + * {{{ + * bin/run-example ml.DeveloperApiExample + * }}} + */ +object DeveloperApiExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("DeveloperApiExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Prepare training data. + // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans + // into SchemaRDDs, where it uses the bean metadata to infer the schema. + val training = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) + + // Create a LogisticRegression instance. This instance is an Estimator. + val lr = new MyLogisticRegression() + // Print out the parameters, documentation, and any default values. + println("MyLogisticRegression parameters:\n" + lr.explainParams() + "\n") + + // We may set parameters using setter methods. + lr.setMaxIter(10) + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + val model = lr.fit(training) + + // Prepare test data. + val test = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) + + // Make predictions on test data. + val sumPredictions: Double = model.transform(test) + .select('features, 'label, 'prediction) + .collect() + .map { case Row(features: Vector, label: Double, prediction: Double) => + prediction + }.sum + assert(sumPredictions == 0.0, + "MyLogisticRegression predicted something other than 0, even though all weights are 0!") + } +} + +/** + * Example of defining a parameter trait for a user-defined type of [[Classifier]]. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +private trait MyLogisticRegressionParams extends ClassifierParams { + + /** param for max number of iterations */ + val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") + def getMaxIter: Int = get(maxIter) +} + +/** + * Example of defining a type of [[Classifier]]. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +private class MyLogisticRegression + extends Classifier[Vector, MyLogisticRegression, MyLogisticRegressionModel] + with MyLogisticRegressionParams { + + setMaxIter(100) // Initialize + + def setMaxIter(value: Int): this.type = set(maxIter, value) + + override def fit(dataset: SchemaRDD, paramMap: ParamMap): MyLogisticRegressionModel = { + // Check schema (types). This allows early failure before running the algorithm. + transformSchema(dataset.schema, paramMap, logging = true) + + // Extract columns from data using helper method. + val oldDataset = extractLabeledPoints(dataset, paramMap) + + // Combine given parameters with the embedded parameters, where the given paramMap overrides + // any embedded settings. + val map = this.paramMap ++ paramMap + + // Do learning to estimate the weight vector. + val numFeatures = oldDataset.take(1)(0).features.size + val weights = Vectors.zeros(numFeatures) // Learning would happen here. + + // Create a model to return. + val lrm = new MyLogisticRegressionModel(this, map, weights) + + // Copy model params. + // An Estimator stores the parameters for the Model it produces, and this copies any relevant + // parameters to the model. + Params.inheritValues(map, this, lrm) + + // Return the learned model. + lrm + } + + /** + * Returns the SQL DataType corresponding to the FeaturesType type parameter. + * This is used by [[ClassifierParams.validateAndTransformSchema()]] to check the input data. + */ + override protected def featuresDataType: DataType = new VectorUDT +} + +/** + * Example of defining a type of [[ClassificationModel]]. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +private class MyLogisticRegressionModel( + override val parent: MyLogisticRegression, + override val fittingParamMap: ParamMap, + val weights: Vector) + extends ClassificationModel[Vector, MyLogisticRegressionModel] + with MyLogisticRegressionParams { + + // This uses the default implementation of transform(), which reads column "features" and outputs + // columns "prediction" and "rawPrediction." + + // This uses the default implementation of predict(), which chooses the label corresponding to + // the maximum value returned by [[predictRaw()]]. + + /** + * Raw prediction for each possible label. + * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives + * a measure of confidence in each possible label (where larger = more confident). + * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * + * @return vector where element i is the raw prediction for label i. + * This raw prediction may be any real number, where a larger value indicates greater + * confidence for that label. + */ + override protected def predictRaw(features: Vector): Vector = { + val margin = BLAS.dot(features, weights) + // There are 2 classes (binary classification), so we return a length-2 vector, + // where index i corresponds to class i (i = 0, 1). + Vectors.dense(-margin, margin) + } + + /** Number of classes the label can take. 2 indicates binary classification. */ + override val numClasses: Int = 2 + + /** + * Create a copy of the model. + * The copy is shallow, except for the embedded paramMap, which gets a deep copy. + * + * This is used for the defaul implementation of [[transform()]]. + */ + override protected def copy(): MyLogisticRegressionModel = { + val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights) + Params.inheritValues(this.paramMap, this, m) + m + } + + /** + * Returns the SQL DataType corresponding to the FeaturesType type parameter. + * This is used by [[ClassifierParams.validateAndTransformSchema()]] to check the input data. + */ + override protected def featuresDataType: DataType = new VectorUDT +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 4d1530cd1349f..325492c5a577e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -72,7 +72,7 @@ object SimpleParamsExample { paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. // One can also combine ParamMaps. - val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Change output column name + val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name val paramMapCombined = paramMap ++ paramMap2 // Now learn a new model using the paramMapCombined parameters. @@ -80,18 +80,18 @@ object SimpleParamsExample { val model2 = lr.fit(training, paramMapCombined) println("Model 2 was fit using parameters: " + model2.fittingParamMap) - // Prepare test documents. - val test = sc.parallelize(Seq( + // Prepare test data. + val test = sparkContext.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) - // Make predictions on test documents using the Transformer.transform() method. + // Make predictions on test data using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. - // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' - // column since we renamed the lr.scoreCol parameter previously. + // Note that model2.transform() outputs a 'myProbability' column instead of the usual + // 'probability' column since we renamed the lr.probabilityCol parameter previously. model2.transform(test) - .select("features", "label", "probability", "prediction") + .select('features, 'label, 'myProbability, 'prediction) .collect() .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index dbbe01dd5ce8e..0a5adaa7fc1ed 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -23,6 +23,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{Row, SQLContext} @BeanInfo @@ -79,10 +80,10 @@ object SimpleTextClassificationPipeline { // Make predictions on test documents. model.transform(test) - .select("id", "text", "score", "prediction") + .select('id, 'text, 'probability, 'prediction) .collect() - .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => - println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => + println("(" + id + ", " + text + ") --> prob=" + prob + ", prediction=" + prediction) } sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala deleted file mode 100644 index 8b6b2f3fa2756..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/LabeledPoint.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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 - -import scala.beans.BeanInfo - -import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.mllib.linalg.Vector - -/** - * :: AlphaComponent :: - * Class that represents an instance (data point) for prediction tasks. - * - * @param label Label to predict - * @param features List of features describing this instance - * @param weight Instance weight - */ -@AlphaComponent -@BeanInfo -case class LabeledPoint(label: Double, features: Vector, weight: Double) { - - /** Constructor which sets instance weight to 1.0 */ - def this(label: Double, features: Vector) = this(label, features, 1.0) - - override def toString: String = { - "(%s,%s,%s)".format(label, features, weight) - } -} - -/** - * :: AlphaComponent :: - */ -@AlphaComponent -object LabeledPoint { - /** Constructor which sets instance weight to 1.0 */ - def apply(label: Double, features: Vector) = new LabeledPoint(label, features, 1.0) -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 2f31beb7303fb..243de234dffdf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,27 +17,56 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.api.java.JavaRDD +import scala.reflect.runtime.universe._ + +import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.rdd.RDD +import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.Star /** + * :: DeveloperApi :: * Params for classification. - * Currently empty, but may add functionality later. */ -private[classification] trait ClassifierParams extends PredictorParams +@DeveloperApi +trait ClassifierParams extends PredictorParams + with HasRawPredictionCol { + + override protected def validateAndTransformSchema( + schema: StructType, + paramMap: ParamMap, + fitting: Boolean, + featuresDataType: DataType): StructType = { + val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) + val map = this.paramMap ++ paramMap + addOutputColumn(parentSchema, map(rawPredictionCol), new VectorUDT) + } +} /** - * Single-label binary or multiclass classification + * :: AlphaComponent :: + * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam Learner Concrete Estimator type + * @tparam M Concrete Model type */ @AlphaComponent -abstract class Classifier[Learner <: Classifier[Learner, M], M <: ClassificationModel[M]] - extends Predictor[Learner, M] +abstract class Classifier[ + FeaturesType, + Learner <: Classifier[FeaturesType, Learner, M], + M <: ClassificationModel[FeaturesType, M]] + extends Predictor[FeaturesType, Learner, M] with ClassifierParams { + setRawPredictionCol("") // Do not output by default + + def setRawPredictionCol(value: String): Learner = + set(rawPredictionCol, value).asInstanceOf[Learner] + // TODO: defaultEvaluator (follow-up PR) } @@ -46,42 +75,130 @@ abstract class Classifier[Learner <: Classifier[Learner, M], M <: Classification * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam M Model type. + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam M Concrete Model type */ @AlphaComponent -abstract class ClassificationModel[M <: ClassificationModel[M]] - extends PredictionModel[M] with ClassifierParams { +abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with ClassifierParams { + + setRawPredictionCol("") // Do not output by default + + def setRawPredictionCol(value: String): M = set(rawPredictionCol, value).asInstanceOf[M] /** Number of classes (values which the label can take). */ def numClasses: Int /** + * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by + * parameters: + * - predicted labels as [[predictionCol]] of type [[Double]] + * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]]. + * + * @param dataset input dataset + * @param paramMap additional parameters, overwrite embedded params + * @return transformed dataset + */ + override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + // This default implementation should be overridden as needed. + + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + + // Prepare model + val tmpModel = if (paramMap.size != 0) { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + tmpModel + } else { + this + } + + val (numColsOutput, outputData) = + ClassificationModel.transformColumnsImpl[FeaturesType](dataset, tmpModel, map) + if (numColsOutput == 0) { + logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") + } + outputData + } + + /** + * :: DeveloperApi :: + * * Predict label for the given features. + * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + * * This default implementation for classification predicts the index of the maximum value * from [[predictRaw()]]. */ - override def predict(features: Vector): Double = { + @DeveloperApi + override protected def predict(features: FeaturesType): Double = { predictRaw(features).toArray.zipWithIndex.maxBy(_._1)._2 } /** + * :: DeveloperApi :: + * * Raw prediction for each possible label. * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives - * a magnitude of confidence in each possible label. + * a measure of confidence in each possible label (where larger = more confident). + * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * * @return vector where element i is the raw prediction for label i. * This raw prediction may be any real number, where a larger value indicates greater * confidence for that label. */ - def predictRaw(features: Vector): Vector + @DeveloperApi + protected def predictRaw(features: FeaturesType): Vector + +} + +private[ml] object ClassificationModel { + + /** + * Added prediction column(s). This is separated from [[ClassificationModel.transform()]] + * since it is used by [[org.apache.spark.ml.classification.ProbabilisticClassificationModel]]. + * @param dataset Input dataset + * @param map Parameter map. This will NOT be merged with the embedded paramMap; the merge + * should already be done. + * @return (number of columns added, transformed dataset) + */ + private[ml] def transformColumnsImpl[FeaturesType]( + dataset: SchemaRDD, + model: ClassificationModel[FeaturesType, _], + map: ParamMap): (Int, SchemaRDD) = { - /** Batch version of [[predictRaw]] */ - def predictRaw(dataset: RDD[Vector]): RDD[Vector] = dataset.map(predictRaw) + import org.apache.spark.sql.catalyst.dsl._ + import dataset.sqlContext._ - /** Java-friendly batch version of [[predictRaw]] */ - def predictRaw(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { - dataset.rdd.map(predictRaw).toJavaRDD() + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + var tmpData = dataset + var numColsOutput = 0 + if (map(model.rawPredictionCol) != "") { + // output raw prediction + val features2raw: FeaturesType => Vector = model.predictRaw + tmpData = tmpData.select(Star(None), + features2raw.call(map(model.featuresCol).attr) as map(model.rawPredictionCol)) + numColsOutput += 1 + if (map(model.predictionCol) != "") { + val raw2pred: Vector => Double = (rawPred) => { + rawPred.toArray.zipWithIndex.maxBy(_._1)._2 + } + tmpData = tmpData.select(Star(None), + raw2pred.call(map(model.rawPredictionCol).attr) as map(model.predictionCol)) + numColsOutput += 1 + } + } else if (map(model.predictionCol) != "") { + // output prediction + val features2pred: FeaturesType => Double = model.predict + tmpData = tmpData.select(Star(None), + features2pred.call(map(model.featuresCol).attr) as map(model.predictionCol)) + numColsOutput += 1 + } + (numColsOutput, tmpData) } - // TODO: accuracy(dataset: RDD[LabeledPoint]): Double (follow-up PR) - } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index b8628d6cb8778..8f07ff6d58edb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -18,12 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.LabeledPoint -import org.apache.spark.ml.impl.estimator.ProbabilisticClassificationModel import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{Vectors, BLAS, Vector} -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{VectorUDT, Vectors, BLAS, Vector} import org.apache.spark.sql._ import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -32,22 +29,8 @@ import org.apache.spark.storage.StorageLevel /** * Params for logistic regression. */ -private[classification] trait LogisticRegressionParams extends ClassifierParams - with HasRegParam with HasMaxIter with HasThreshold with HasScoreCol { - - override protected def validateAndTransformSchema( - schema: StructType, - paramMap: ParamMap, - fitting: Boolean): StructType = { - val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting) - val map = this.paramMap ++ paramMap - val fieldNames = parentSchema.fieldNames - require(!fieldNames.contains(map(scoreCol)), s"Score column ${map(scoreCol)} already exists.") - val outputFields = parentSchema.fields ++ Seq( - StructField(map(scoreCol), DoubleType, nullable = false)) - StructType(outputFields) - } -} +private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams + with HasRegParam with HasMaxIter with HasThreshold /** @@ -56,7 +39,8 @@ private[classification] trait LogisticRegressionParams extends ClassifierParams * Currently, this class only supports binary classification. */ @AlphaComponent -class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressionModel] +class LogisticRegression + extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { setRegParam(0.1) @@ -66,44 +50,37 @@ class LogisticRegression extends Classifier[LogisticRegression, LogisticRegressi def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) def setThreshold(value: Double): this.type = set(threshold, value) - def setScoreCol(value: String): this.type = set(scoreCol, value) - /** - * Same as [[fit()]], but using strong types. - * NOTE: This does NOT support instance weights. - * @param dataset Training data. Instance weights are ignored. - * @param paramMap Parameters for training. - * These values override any specified in this Estimator's embedded ParamMap. - */ - override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LogisticRegressionModel = { + override def fit(dataset: SchemaRDD, paramMap: ParamMap): LogisticRegressionModel = { + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + + // Extract columns from data. If dataset is persisted, do not persist oldDataset. + val oldDataset = extractLabeledPoints(dataset, paramMap) val map = this.paramMap ++ paramMap - val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => - org.apache.spark.mllib.regression.LabeledPoint(label, features) - } - // If dataset is persisted, do not persist oldDataset. val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) } + + // Train model val lr = new LogisticRegressionWithLBFGS lr.optimizer .setRegParam(map(regParam)) .setNumIterations(map(maxIter)) - val model = lr.run(oldDataset) - val lrm = new LogisticRegressionModel(this, map, model.weights, model.intercept) + val oldModel = lr.run(oldDataset) + val lrm = new LogisticRegressionModel(this, map, oldModel.weights, oldModel.intercept) + if (handlePersistence) { oldDataset.unpersist() } + + // copy model params + Params.inheritValues(map, this, lrm) lrm } - /** - * Same as [[fit()]], but using strong types. - * NOTE: This does NOT support instance weights. - * @param dataset Training data. Instance weights are ignored. - */ - override def train(dataset: RDD[LabeledPoint]): LogisticRegressionModel = - train(dataset, new ParamMap()) // Override documentation + override protected def featuresDataType: DataType = new VectorUDT } @@ -117,14 +94,12 @@ class LogisticRegressionModel private[ml] ( override val fittingParamMap: ParamMap, val weights: Vector, val intercept: Double) - extends ClassificationModel[LogisticRegressionModel] - with ProbabilisticClassificationModel + extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with LogisticRegressionParams { setThreshold(0.5) def setThreshold(value: Double): this.type = set(threshold, value) - def setScoreCol(value: String): this.type = set(scoreCol, value) private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept @@ -135,6 +110,7 @@ class LogisticRegressionModel private[ml] ( 1.0 / (1.0 + math.exp(-m)) } +/* override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap @@ -145,35 +121,98 @@ class LogisticRegressionModel private[ml] ( val t = map(threshold) val predictFunction: Double => Double = (score) => { if (score > t) 1.0 else 0.0 } dataset - .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) - .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol))) + .select($"*", callUDF(scoreFunction, col(map(featuresCol))).as(map(scoreCol))) + .select($"*", callUDF(predictFunction, col(map(scoreCol))).as(map(predictionCol))) +*/ + override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + + import dataset.sqlContext._ + val map = this.paramMap ++ paramMap + + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + // rawPrediction (-margin, margin) + // probability (1.0-score, score) + // prediction (max margin) + var tmpData = dataset + var numColsOutput = 0 + if (map(rawPredictionCol) != "") { + val features2raw: Vector => Vector = predictRaw + tmpData = tmpData.select(Star(None), + features2raw.call(map(featuresCol).attr) as map(rawPredictionCol)) + numColsOutput += 1 + } + if (map(probabilityCol) != "") { + if (map(rawPredictionCol) != "") { + val raw2prob: Vector => Vector = (rawPreds) => { + val prob1 = 1.0 / 1.0 + math.exp(-rawPreds(1)) + Vectors.dense(1.0 - prob1, prob1) + } + tmpData = tmpData.select(Star(None), + raw2prob.call(map(rawPredictionCol).attr) as map(probabilityCol)) + } else { + val features2prob: Vector => Vector = predictProbabilities + tmpData = tmpData.select(Star(None), + features2prob.call(map(featuresCol).attr) as map(probabilityCol)) + } + numColsOutput += 1 + } + if (map(predictionCol) != "") { + val t = map(threshold) + if (map(probabilityCol) != "") { + val predict: Vector => Double = (probs) => { + if (probs(1) > t) 1.0 else 0.0 + } + tmpData = tmpData.select(Star(None), + predict.call(map(probabilityCol).attr) as map(predictionCol)) + } else if (map(rawPredictionCol) != "") { + val predict: Vector => Double = (rawPreds) => { + val prob1 = 1.0 / 1.0 + math.exp(-rawPreds(1)) + if (prob1 > t) 1.0 else 0.0 + } + tmpData = tmpData.select(Star(None), + predict.call(map(rawPredictionCol).attr) as map(predictionCol)) + } else { + val predict: Vector => Double = this.predict + tmpData = tmpData.select(Star(None), + predict.call(map(featuresCol).attr) as map(predictionCol)) + } + numColsOutput += 1 + } + if (numColsOutput == 0) { + this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + + " since no output columns were set.") + } + tmpData } override val numClasses: Int = 2 - // TODO: Override batch predict() for efficiency. - /** * Predict label for the given feature vector. * The behavior of this can be adjusted using [[threshold]]. */ - override def predict(features: Vector): Double = { + override protected def predict(features: Vector): Double = { if (score(features) > paramMap(threshold)) 1 else 0 } - override def predictProbabilities(features: Vector): Vector = { + override protected def predictProbabilities(features: Vector): Vector = { val s = score(features) - Vectors.dense(Array(1.0 - s, s)) + Vectors.dense(1.0 - s, s) } - override def predictRaw(features: Vector): Vector = { + override protected def predictRaw(features: Vector): Vector = { val m = margin(features) - Vectors.dense(Array(-m, m)) + Vectors.dense(-m, m) } - private[ml] override def copy(): LogisticRegressionModel = { + override protected def copy(): LogisticRegressionModel = { val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) Params.inheritValues(this.paramMap, this, m) m } + + override protected def featuresDataType: DataType = new VectorUDT } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala new file mode 100644 index 0000000000000..41f9b9601a00b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -0,0 +1,145 @@ +/* + * 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 scala.reflect.runtime.universe._ + +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.Star + +/** + * Params for probabilistic classification. + */ +private[classification] trait ProbabilisticClassifierParams + extends ClassifierParams with HasProbabilityCol { + + override protected def validateAndTransformSchema( + schema: StructType, + paramMap: ParamMap, + fitting: Boolean, + featuresDataType: DataType): StructType = { + val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) + val map = this.paramMap ++ paramMap + addOutputColumn(parentSchema, map(probabilityCol), new VectorUDT) + } +} + +/** + * :: AlphaComponent :: + * Single-label binary or multiclass classifier which can output class conditional probabilities. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam Learner Concrete Estimator type + * @tparam M Concrete Model type + */ +@AlphaComponent +abstract class ProbabilisticClassifier[ + FeaturesType, + Learner <: ProbabilisticClassifier[FeaturesType, Learner, M], + M <: ProbabilisticClassificationModel[FeaturesType, M]] + extends Classifier[FeaturesType, Learner, M] with ProbabilisticClassifierParams { + + setProbabilityCol("") // Do not output by default + + def setProbabilityCol(value: String): Learner = set(probabilityCol, value).asInstanceOf[Learner] +} + +/** + * :: AlphaComponent :: + * Model produced by a [[ProbabilisticClassifier]]. + * Classes are indexed {0, 1, ..., numClasses - 1}. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam M Concrete Model type + */ +@AlphaComponent +abstract class ProbabilisticClassificationModel[ + FeaturesType, + M <: ProbabilisticClassificationModel[FeaturesType, M]] + extends ClassificationModel[FeaturesType, M] with ProbabilisticClassifierParams { + + setProbabilityCol("") // Do not output by default + + def setProbabilityCol(value: String): M = set(probabilityCol, value).asInstanceOf[M] + + /** + * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by + * parameters: + * - predicted labels as [[predictionCol]] of type [[Double]] + * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]] + * - probability of each class as [[probabilityCol]] of type [[Vector]]. + * + * @param dataset input dataset + * @param paramMap additional parameters, overwrite embedded params + * @return transformed dataset + */ + override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + // This default implementation should be overridden as needed. + import dataset.sqlContext._ + import org.apache.spark.sql.catalyst.dsl._ + + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + + // Prepare model + val tmpModel = if (paramMap.size != 0) { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + tmpModel + } else { + this + } + + val (numColsOutput, outputData) = + ClassificationModel.transformColumnsImpl[FeaturesType](dataset, tmpModel, map) + + // Output selected columns only. + if (map(probabilityCol) != "") { + // output probabilities + val features2probs: FeaturesType => Vector = (features) => { + tmpModel.predictProbabilities(features) + } + outputData.select(Star(None), + features2probs.call(map(featuresCol).attr) as map(probabilityCol)) + } else { + if (numColsOutput == 0) { + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") + } + outputData + } + } + + /** + * :: DeveloperApi :: + * + * Predict the probability of each class given the features. + * These predictions are also called class conditional probabilities. + * + * WARNING: Not all models output well-calibrated probability estimates! These probabilities + * should be treated as confidences, not precise probabilities. + * + * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + */ + @DeveloperApi + protected def predictProbabilities(features: FeaturesType): Vector +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 1979ab9eb6516..9dcd6c0682256 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -22,6 +22,7 @@ import org.apache.spark.ml._ import org.apache.spark.ml.param._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.types.DoubleType /** @@ -30,7 +31,7 @@ import org.apache.spark.sql.types.DoubleType */ @AlphaComponent class BinaryClassificationEvaluator extends Evaluator with Params - with HasScoreCol with HasLabelCol { + with HasRawPredictionCol with HasLabelCol { /** param for metric name in evaluation */ val metricName: Param[String] = new Param(this, "metricName", @@ -38,23 +39,20 @@ class BinaryClassificationEvaluator extends Evaluator with Params def getMetricName: String = get(metricName) def setMetricName(value: String): this.type = set(metricName, value) - def setScoreCol(value: String): this.type = set(scoreCol, value) + def setScoreCol(value: String): this.type = set(rawPredictionCol, value) def setLabelCol(value: String): this.type = set(labelCol, value) override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { val map = this.paramMap ++ paramMap val schema = dataset.schema - val scoreType = schema(map(scoreCol)).dataType - require(scoreType == DoubleType, - s"Score column ${map(scoreCol)} must be double type but found $scoreType") - val labelType = schema(map(labelCol)).dataType - require(labelType == DoubleType, - s"Label column ${map(labelCol)} must be double type but found $labelType") + checkInputColumn(schema, map(rawPredictionCol), new VectorUDT) + checkInputColumn(schema, map(labelCol), DoubleType) - val scoreAndLabels = dataset.select(map(scoreCol), map(labelCol)) - .map { case Row(score: Double, label: Double) => - (score, label) + // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. + val scoreAndLabels = dataset.select(map(rawPredictionCol).attr, map(labelCol).attr) + .map { case Row(rawPrediction: Vector, label: Double) => + (rawPrediction(1), label) } val metrics = new BinaryClassificationMetrics(scoreAndLabels) val metric = map(metricName) match { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index e622a5cf9e6f3..0b1f90daa7d8e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -29,11 +29,11 @@ import org.apache.spark.sql.types.{DataType, StringType, ArrayType} @AlphaComponent class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { - protected override def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { + override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { _.toLowerCase.split("\\s") } - protected override def validateInputType(inputType: DataType): Unit = { + override protected def validateInputType(inputType: DataType): Unit = { require(inputType == StringType, s"Input type must be string type but got $inputType.") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 48cecfefd4c07..35ca5a0bcbe00 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -17,15 +17,23 @@ package org.apache.spark.ml.impl.estimator -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.ml.{Estimator, LabeledPoint, Model} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star -private[ml] trait PredictorParams extends Params + +/** + * :: DeveloperApi :: + * + * Trait for parameters for prediction (regression and classification). + */ +@DeveloperApi +trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol { /** @@ -33,33 +41,41 @@ private[ml] trait PredictorParams extends Params * @param schema input schema * @param paramMap additional parameters * @param fitting whether this is in fitting + * @param featuresDataType SQL DataType for FeaturesType. + * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. * @return output schema */ protected def validateAndTransformSchema( schema: StructType, paramMap: ParamMap, - fitting: Boolean): StructType = { + fitting: Boolean, + featuresDataType: DataType): StructType = { val map = this.paramMap ++ paramMap - val featuresType = schema(map(featuresCol)).dataType - // TODO: Support casting Array[Double] and Array[Float] to Vector. - require(featuresType.isInstanceOf[VectorUDT], - s"Features column ${map(featuresCol)} must be Vector types" + - s" but was actually $featuresType.") + // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector + checkInputColumn(schema, map(featuresCol), featuresDataType) if (fitting) { - val labelType = schema(map(labelCol)).dataType - require(labelType == DoubleType || labelType == IntegerType, - s"Cannot convert label column ${map(labelCol)} of type $labelType to a Double column.") + // TODO: Allow other numeric types + checkInputColumn(schema, map(labelCol), DoubleType) } - val fieldNames = schema.fieldNames - require(!fieldNames.contains(map(predictionCol)), - s"Prediction column ${map(predictionCol)} already exists.") - val outputFields = schema.fields ++ Seq( - StructField(map(predictionCol), DoubleType, nullable = false)) - StructType(outputFields) + addOutputColumn(schema, map(predictionCol), DoubleType) } } -private[ml] abstract class Predictor[Learner <: Predictor[Learner, M], M <: PredictionModel[M]] +/** + * Abstraction for prediction problems (regression and classification). + * + * @tparam FeaturesType Type of features. + * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * @tparam Learner Specialization of this class. If you subclass this type, use this type + * parameter to specify the concrete type. + * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type + * parameter to specify the concrete type for the corresponding model. + */ +@DeveloperApi +abstract class Predictor[ + FeaturesType, + Learner <: Predictor[FeaturesType, Learner, M], + M <: PredictionModel[FeaturesType, M]] extends Estimator[M] with PredictorParams { // TODO: Eliminate asInstanceOf and see if that works. @@ -67,6 +83,8 @@ private[ml] abstract class Predictor[Learner <: Predictor[Learner, M], M <: Pred def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] + /* + // This will be useful for boosting. protected def selectLabelColumn(dataset: SchemaRDD, paramMap: ParamMap): RDD[Double] = { import dataset.sqlContext._ val map = this.paramMap ++ paramMap @@ -75,113 +93,109 @@ private[ml] abstract class Predictor[Learner <: Predictor[Learner, M], M <: Pred case Row(label: Int) => label.toDouble } } + */ + + /** + * :: DeveloperApi :: + * + * Returns the SQL DataType corresponding to the FeaturesType type parameter. + * + * This is used by [[validateAndTransformSchema()]]. + * This workaround is needed since SQL has different APIs for Scala and Java. + */ + @DeveloperApi + protected def featuresDataType: DataType private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = true) + validateAndTransformSchema(schema, paramMap, fitting = true, featuresDataType) } - override def fit(dataset: SchemaRDD, paramMap: ParamMap): M = { - transformSchema(dataset.schema, paramMap, logging = true) + /** + * Extract [[labelCol]] and [[featuresCol]] from the given dataset, + * and put it in an RDD with strong types. + */ + protected def extractLabeledPoints(dataset: SchemaRDD, paramMap: ParamMap): RDD[LabeledPoint] = { import dataset.sqlContext._ val map = this.paramMap ++ paramMap - val instances = dataset.select(map(labelCol).attr, map(featuresCol).attr) + dataset.select(map(labelCol).attr, map(featuresCol).attr) .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } - val model = train(instances, map) - // copy model params - Params.inheritValues(map, this, model) - model } - - /** - * Same as [[fit()]], but using strong types. - * - * @param dataset Training data - * @param paramMap Parameters for training. - * These values override any specified in this Estimator's embedded ParamMap. - */ - def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): M - - /** - * Same as [[fit()]], but using strong types. - * @param dataset Training data - */ - def train(dataset: RDD[LabeledPoint]): M = train(dataset, new ParamMap()) - - /** Java-friendly version of [[train()]]. */ - def train(dataset: JavaRDD[LabeledPoint], paramMap: ParamMap): M = train(dataset.rdd, paramMap) - - /** Java-friendly version of [[train()]]. */ - def train(dataset: JavaRDD[LabeledPoint]): M = train(dataset.rdd) } -private[ml] abstract class PredictionModel[M <: PredictionModel[M]] +private[ml] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { def setFeaturesCol(value: String): M = set(featuresCol, value).asInstanceOf[M] def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] + /** + * :: DeveloperApi :: + * + * Returns the SQL DataType corresponding to the FeaturesType type parameter. + * + * This is used by [[validateAndTransformSchema()]]. + * This workaround is needed since SQL has different APIs for Scala and Java. + */ + @DeveloperApi + protected def featuresDataType: DataType + private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = false) + validateAndTransformSchema(schema, paramMap, fitting = false, featuresDataType) } /** - * Transforms dataset by reading from [[featuresCol]], calling [[predict( )]], and storing + * Transforms dataset by reading from [[featuresCol]], calling [[predict()]], and storing * the predictions as a new column [[predictionCol]]. - * This default implementation should be overridden as needed. + * * @param dataset input dataset * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset with [[predictionCol]] of type [[Double]] */ override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + // This default implementation should be overridden as needed. import org.apache.spark.sql.catalyst.dsl._ import dataset.sqlContext._ + // Check schema transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap - val tmpModel = this.copy() - Params.inheritValues(paramMap, parent, tmpModel) - val pred: Vector => Double = (features) => { - tmpModel.predict(features) + + // Prepare model + val tmpModel = if (paramMap.size != 0) { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + tmpModel + } else { + this } - dataset.select(Star(None), pred.call(map(featuresCol).attr) as map(predictionCol)) - } - /** - * Strongly typed version of [[transform()]]. - * Default implementation using single-instance predict(). - * - * Developers should override this for efficiency. E.g., this does not broadcast the model. - */ - def predict(dataset: RDD[Vector], paramMap: ParamMap): RDD[Double] = { - val tmpModel = this.copy() - Params.inheritValues(paramMap, parent, tmpModel) - dataset.map(tmpModel.predict) + if (map(predictionCol) != "") { + val pred: FeaturesType => Double = (features) => { + tmpModel.predict(features) + } + dataset.select(Star(None), pred.call(map(featuresCol).attr) as map(predictionCol)) + } else { + this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + + " since no output columns were set.") + dataset + } } - /** Strongly typed version of [[transform()]]. */ - def predict(dataset: RDD[Vector]): RDD[Double] = predict(dataset, new ParamMap) - /** + * :: DeveloperApi :: + * * Predict label for the given features. + * This internal method is used to implement [[transform()]] and output [[predictionCol]]. */ - def predict(features: Vector): Double - - /** Java-friendly version of [[predict()]]. */ - def predict(dataset: JavaRDD[Vector], paramMap: ParamMap): JavaRDD[java.lang.Double] = { - predict(dataset.rdd, paramMap).map(_.asInstanceOf[java.lang.Double]).toJavaRDD() - } - - /** Java-friendly version of [[predict()]]. */ - def predict(dataset: JavaRDD[Vector]): JavaRDD[java.lang.Double] = { - predict(dataset.rdd, new ParamMap).map(_.asInstanceOf[java.lang.Double]).toJavaRDD() - } + @DeveloperApi + protected def predict(features: FeaturesType): Double /** * Create a copy of the model. * The copy is shallow, except for the embedded paramMap, which gets a deep copy. */ - private[ml] def copy(): M + protected def copy(): M } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala deleted file mode 100644 index e534a8c264bb3..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/ProbabilisticClassificationModel.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.impl.estimator - -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.rdd.RDD - -/** - * Trait for a [[org.apache.spark.ml.classification.ClassificationModel]] which can output - * class conditional probabilities. - */ -private[ml] trait ProbabilisticClassificationModel { - - /** - * Predict the probability of each class given the features. - * These predictions are also called class conditional probabilities. - * - * WARNING: Not all models output well-calibrated probability estimates! These probabilities - * should be treated as confidences, not precise probabilities. - */ - def predictProbabilities(features: Vector): Vector - - /** Batch version of [[predictProbabilities()]] */ - def predictProbabilities(features: RDD[Vector]): RDD[Vector] = features.map(predictProbabilities) - - /** Java-friendly batch version of [[predictProbabilities()]] */ - def predictProbabilities(features: JavaRDD[Vector]): JavaRDD[Vector] = { - features.rdd.map(predictProbabilities).toJavaRDD() - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 465bfa9099c1d..b3e3e58665fef 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -19,11 +19,14 @@ package org.apache.spark.ml.param import scala.annotation.varargs import scala.collection.mutable +import scala.reflect.runtime.universe._ import java.lang.reflect.Modifier -import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} import org.apache.spark.ml.Identifiable +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.ScalaReflection /** * :: AlphaComponent :: @@ -158,7 +161,7 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter in the embedded param map. */ - private[ml] def set[T](param: Param[T], value: T): this.type = { + protected def set[T](param: Param[T], value: T): this.type = { require(param.parent.eq(this)) paramMap.put(param.asInstanceOf[Param[Any]], value) this @@ -174,7 +177,7 @@ trait Params extends Identifiable with Serializable { /** * Gets the value of a parameter in the embedded param map. */ - private[ml] def get[T](param: Param[T]): T = { + protected def get[T](param: Param[T]): T = { require(param.parent.eq(this)) paramMap(param) } @@ -183,9 +186,38 @@ trait Params extends Identifiable with Serializable { * Internal param map. */ protected val paramMap: ParamMap = ParamMap.empty + + /** + * Check whether the given schema contains an input column. + * @param colName Parameter name for the input column. + * @param dataType SQL DataType of the input column. + */ + protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { + val actualDataType = schema(colName).dataType + require(actualDataType.equals(dataType), + s"Input column $colName must be of type $dataType" + + s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + } + + protected def addOutputColumn( + schema: StructType, + colName: String, + dataType: DataType): StructType = { + if (colName.length == 0) return schema + val fieldNames = schema.fieldNames + require(!fieldNames.contains(colName), s"Prediction column $colName already exists.") + val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) + StructType(outputFields) + } } -private[ml] object Params { +/** + * :: DeveloperApi :: + * + * Helper functionality for developers. + */ +@DeveloperApi +object Params { /** * Copies parameter values from the parent estimator to the child model it produced. @@ -310,6 +342,11 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten ParamPair(param, value) } } + + /** + * Number of param pairs in this set. + */ + def size: Int = map.size } object ParamMap { diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index ef141d3eb2b06..bf336f3f7173b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -17,6 +17,10 @@ package org.apache.spark.ml.param +/* NOTE TO DEVELOPERS: + * If you add these parameter traits into your algorithm, you need to add a setter method as well. + */ + private[ml] trait HasRegParam extends Params { /** param for regularization parameter */ val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") @@ -42,12 +46,6 @@ private[ml] trait HasLabelCol extends Params { def getLabelCol: String = get(labelCol) } -private[ml] trait HasScoreCol extends Params { - /** param for score column name */ - val scoreCol: Param[String] = new Param(this, "scoreCol", "score column name", Some("score")) - def getScoreCol: String = get(scoreCol) -} - private[ml] trait HasPredictionCol extends Params { /** param for prediction column name */ val predictionCol: Param[String] = @@ -55,6 +53,22 @@ private[ml] trait HasPredictionCol extends Params { def getPredictionCol: String = get(predictionCol) } +private[ml] trait HasRawPredictionCol extends Params { + /** param for raw prediction column name */ + val rawPredictionCol: Param[String] = + new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", + Some("rawPrediction")) + def getRawPredictionCol: String = get(rawPredictionCol) +} + +private[ml] trait HasProbabilityCol extends Params { + /** param for predicted class conditional probabilities column name */ + val probabilityCol: Param[String] = + new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", + Some("probability")) + def getProbabilityCol: String = get(probabilityCol) +} + private[ml] trait HasThreshold extends Params { /** param for threshold in (binary) prediction */ val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index e6abe9b404808..3ff7107221763 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -18,11 +18,10 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.LabeledPoint import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} -import org.apache.spark.mllib.linalg.{BLAS, Vector} +import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ import org.apache.spark.storage.StorageLevel /** @@ -36,7 +35,7 @@ private[regression] trait LinearRegressionParams extends RegressorParams * Logistic regression. */ @AlphaComponent -class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel] +class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] with LinearRegressionParams { setRegParam(0.1) @@ -45,41 +44,36 @@ class LinearRegression extends Regressor[LinearRegression, LinearRegressionModel def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) - /** - * Same as [[fit()]], but using strong types. - * NOTE: This does NOT support instance weights. - * @param dataset Training data. Instance weights are ignored. - * @param paramMap Parameters for training. - * These values override any specified in this Estimator's embedded ParamMap. - */ - override def train(dataset: RDD[LabeledPoint], paramMap: ParamMap): LinearRegressionModel = { + override def fit(dataset: SchemaRDD, paramMap: ParamMap): LinearRegressionModel = { + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + + // Extract columns from data. If dataset is persisted, do not persist oldDataset. + val oldDataset = extractLabeledPoints(dataset, paramMap) val map = this.paramMap ++ paramMap - val oldDataset = dataset.map { case LabeledPoint(label: Double, features: Vector, weight) => - org.apache.spark.mllib.regression.LabeledPoint(label, features) - } - val handlePersistence = oldDataset.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) } + + // Train model val lr = new LinearRegressionWithSGD() lr.optimizer .setRegParam(map(regParam)) .setNumIterations(map(maxIter)) val model = lr.run(oldDataset) val lrm = new LinearRegressionModel(this, map, model.weights, model.intercept) + if (handlePersistence) { oldDataset.unpersist() } + + // copy model params + Params.inheritValues(map, this, lrm) lrm } - /** - * Same as [[fit()]], but using strong types. - * NOTE: This does NOT support instance weights. - * @param dataset Training data. Instance weights are ignored. - */ - override def train(dataset: RDD[LabeledPoint]): LinearRegressionModel = - train(dataset, new ParamMap()) // Override documentation + override protected def featuresDataType: DataType = new VectorUDT } /** @@ -92,16 +86,18 @@ class LinearRegressionModel private[ml] ( override val fittingParamMap: ParamMap, val weights: Vector, val intercept: Double) - extends RegressionModel[LinearRegressionModel] + extends RegressionModel[Vector, LinearRegressionModel] with LinearRegressionParams { - override def predict(features: Vector): Double = { + override protected def predict(features: Vector): Double = { BLAS.dot(features, weights) + intercept } - private[ml] override def copy(): LinearRegressionModel = { + override protected def copy(): LinearRegressionModel = { val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) Params.inheritValues(this.paramMap, this, m) m } + + override protected def featuresDataType: DataType = new VectorUDT } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index 78086fe16fd60..5f10344456a10 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -17,23 +17,31 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.mllib.linalg.Vector /** + * :: DeveloperApi :: * Params for regression. * Currently empty, but may add functionality later. */ -private[regression] trait RegressorParams extends PredictorParams +@DeveloperApi +trait RegressorParams extends PredictorParams /** * :: AlphaComponent :: * Single-label regression + * + * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] + * @tparam Learner Concrete Estimator type + * @tparam M Concrete Model type */ @AlphaComponent -abstract class Regressor[Learner <: Regressor[Learner, M], M <: RegressionModel[M]] - extends Predictor[Learner, M] +abstract class Regressor[ + FeaturesType, + Learner <: Regressor[FeaturesType, Learner, M], + M <: RegressionModel[FeaturesType, M]] + extends Predictor[FeaturesType, Learner, M] with RegressorParams { // TODO: defaultEvaluator (follow-up PR) @@ -42,15 +50,21 @@ abstract class Regressor[Learner <: Regressor[Learner, M], M <: RegressionModel[ /** * :: AlphaComponent :: * Model produced by a [[Regressor]]. - * @tparam M Model type. + * + * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] + * @tparam M Concrete Model type. */ @AlphaComponent -abstract class RegressionModel[M <: RegressionModel[M]] - extends PredictionModel[M] with RegressorParams { +abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with RegressorParams { /** + * :: DeveloperApi :: + * * Predict real-valued label for the given features. + * This internal method is used to implement [[transform()]] and output [[predictionCol]]. */ - def predict(features: Vector): Double + @DeveloperApi + protected def predict(features: FeaturesType): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 77785bdbd03d9..66ee01cdc0d34 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -113,7 +113,7 @@ sealed trait Vector extends Serializable { * User-defined type for [[Vector]] which allows easy interaction with SQL * via [[org.apache.spark.sql.DataFrame]]. */ -private[spark] class VectorUDT extends UserDefinedType[Vector] { +class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { // type: 0 = sparse, 1 = dense @@ -169,6 +169,13 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { override def pyUDT: String = "pyspark.mllib.linalg.VectorUDT" override def userClass: Class[Vector] = classOf[Vector] + + override def equals(o: Any): Boolean = { + o match { + case v: VectorUDT => true + case _ => false + } + } } /** From 934f97b7c876fe704dbe8a55d2b8bcff798b6b59 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 30 Dec 2014 14:44:37 -0800 Subject: [PATCH 17/30] Fixed bugs from previous commit. --- .../examples/ml/JavaSimpleParamsExample.java | 2 +- .../examples/ml/DeveloperApiExample.scala | 2 - .../examples/ml/SimpleParamsExample.scala | 2 +- .../spark/ml/classification/Classifier.scala | 6 -- .../classification/LogisticRegression.scala | 4 +- .../ProbabilisticClassifier.scala | 10 +-- .../BinaryClassificationEvaluator.scala | 1 + .../spark/ml/impl/estimator/Predictor.scala | 19 ++++- .../org/apache/spark/ml/param/params.scala | 4 +- .../ml/regression/LinearRegression.scala | 4 +- .../spark/ml/regression/Regressor.scala | 2 + .../spark/ml/JavaLabeledPointSuite.java | 78 ------------------- .../apache/spark/ml/LabeledPointSuite.scala | 59 -------------- .../LogisticRegressionSuite.scala | 20 ++--- .../ml/regression/LinearRegressionSuite.scala | 2 +- 15 files changed, 43 insertions(+), 172 deletions(-) delete mode 100644 mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java delete mode 100644 mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index 3f4f42594059a..98677d0a4a67b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -102,7 +102,7 @@ public static void main(String[] args) { // 'probability' column since we renamed the lr.probabilityCol parameter previously. model2.transform(test).registerTempTable("results"); DataFrame results = - jsql.sql("SELECT features, label, probability, prediction FROM results"); + jsql.sql("SELECT features, label, myProbability, prediction FROM results"); for (Row r: results.collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 2f1de5c58ed1e..deffce192b2b4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -43,8 +43,6 @@ object DeveloperApiExample { import sqlContext._ // Prepare training data. - // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans - // into SchemaRDDs, where it uses the bean metadata to infer the schema. val training = sparkContext.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 325492c5a577e..80d130728c85f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -93,7 +93,7 @@ object SimpleParamsExample { model2.transform(test) .select('features, 'label, 'myProbability, 'prediction) .collect() - .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => + .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 243de234dffdf..34e4b6fe298d1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,8 +17,6 @@ package org.apache.spark.ml.classification -import scala.reflect.runtime.universe._ - import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} @@ -62,8 +60,6 @@ abstract class Classifier[ extends Predictor[FeaturesType, Learner, M] with ClassifierParams { - setRawPredictionCol("") // Do not output by default - def setRawPredictionCol(value: String): Learner = set(rawPredictionCol, value).asInstanceOf[Learner] @@ -82,8 +78,6 @@ abstract class Classifier[ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { - setRawPredictionCol("") // Do not output by default - def setRawPredictionCol(value: String): M = set(rawPredictionCol, value).asInstanceOf[M] /** Number of classes (values which the label can take). */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 8f07ff6d58edb..2653af994d7c6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{VectorUDT, Vectors, BLAS, Vector} +import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT, Vectors} import org.apache.spark.sql._ import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -35,6 +35,7 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas /** * :: AlphaComponent :: + * * Logistic regression. * Currently, this class only supports binary classification. */ @@ -86,6 +87,7 @@ class LogisticRegression /** * :: AlphaComponent :: + * * Model produced by [[LogisticRegression]]. */ @AlphaComponent diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 41f9b9601a00b..b4da63111f830 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -17,8 +17,6 @@ package org.apache.spark.ml.classification -import scala.reflect.runtime.universe._ - import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} @@ -42,8 +40,10 @@ private[classification] trait ProbabilisticClassifierParams } } + /** * :: AlphaComponent :: + * * Single-label binary or multiclass classifier which can output class conditional probabilities. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] @@ -57,13 +57,13 @@ abstract class ProbabilisticClassifier[ M <: ProbabilisticClassificationModel[FeaturesType, M]] extends Classifier[FeaturesType, Learner, M] with ProbabilisticClassifierParams { - setProbabilityCol("") // Do not output by default - def setProbabilityCol(value: String): Learner = set(probabilityCol, value).asInstanceOf[Learner] } + /** * :: AlphaComponent :: + * * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * @@ -76,8 +76,6 @@ abstract class ProbabilisticClassificationModel[ M <: ProbabilisticClassificationModel[FeaturesType, M]] extends ClassificationModel[FeaturesType, M] with ProbabilisticClassifierParams { - setProbabilityCol("") // Do not output by default - def setProbabilityCol(value: String): M = set(probabilityCol, value).asInstanceOf[M] /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 9dcd6c0682256..af141e95ee9e0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.types.DoubleType /** * :: AlphaComponent :: + * * Evaluator for binary classification, which expects two input columns: score and label. */ @AlphaComponent diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 35ca5a0bcbe00..b39875197f38d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.impl.estimator -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ import org.apache.spark.mllib.linalg.Vector @@ -62,6 +62,8 @@ trait PredictorParams extends Params } /** + * :: AlphaComponent :: + * * Abstraction for prediction problems (regression and classification). * * @tparam FeaturesType Type of features. @@ -71,7 +73,7 @@ trait PredictorParams extends Params * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. */ -@DeveloperApi +@AlphaComponent abstract class Predictor[ FeaturesType, Learner <: Predictor[FeaturesType, Learner, M], @@ -124,7 +126,18 @@ abstract class Predictor[ } } -private[ml] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] +/** + * :: AlphaComponent :: + * + * Abstraction for a model for prediction tasks (regression and classification). + * + * @tparam FeaturesType Type of features. + * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type + * parameter to specify the concrete type for the corresponding model. + */ +@AlphaComponent +abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { def setFeaturesCol(value: String): M = set(featuresCol, value).asInstanceOf[M] diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index b3e3e58665fef..0358309991655 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -19,14 +19,12 @@ package org.apache.spark.ml.param import scala.annotation.varargs import scala.collection.mutable -import scala.reflect.runtime.universe._ import java.lang.reflect.Modifier -import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.Identifiable import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.ScalaReflection /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 3ff7107221763..8ac2738bfe5b5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -32,7 +32,8 @@ private[regression] trait LinearRegressionParams extends RegressorParams /** * :: AlphaComponent :: - * Logistic regression. + * + * Linear regression. */ @AlphaComponent class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] @@ -78,6 +79,7 @@ class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegress /** * :: AlphaComponent :: + * * Model produced by [[LinearRegression]]. */ @AlphaComponent diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index 5f10344456a10..dca849f44270f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -30,6 +30,7 @@ trait RegressorParams extends PredictorParams /** * :: AlphaComponent :: + * * Single-label regression * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] @@ -49,6 +50,7 @@ abstract class Regressor[ /** * :: AlphaComponent :: + * * Model produced by a [[Regressor]]. * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java deleted file mode 100644 index 878ad5a7baed5..0000000000000 --- a/mllib/src/test/java/org/apache/spark/ml/JavaLabeledPointSuite.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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; - -import java.util.List; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import com.google.common.collect.Lists; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; - -/** - * Test {@link LabeledPoint} in Java - */ -public class JavaLabeledPointSuite { - - private transient JavaSparkContext jsc; - private transient JavaSQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLabeledPointSuite"); - jsql = new JavaSQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - - @Test - public void labeledPointDefaultWeight() { - double label = 1.0; - Vector features = Vectors.dense(1.0, 2.0, 3.0); - LabeledPoint lp1 = new LabeledPoint(label, features); - LabeledPoint lp2 = new LabeledPoint(label, features, 1.0); - assert(lp1.equals(lp2)); - } - - @Test - public void labeledPointSchemaRDD() { - List arr = Lists.newArrayList( - new LabeledPoint(0.0, Vectors.dense(1.0, 2.0, 3.0)), - new LabeledPoint(1.0, Vectors.dense(1.1, 2.1, 3.1)), - new LabeledPoint(0.0, Vectors.dense(1.2, 2.2, 3.2)), - new LabeledPoint(1.0, Vectors.dense(1.3, 2.3, 3.3))); - JavaRDD rdd = jsc.parallelize(arr); - JavaSchemaRDD schemaRDD = jsql.applySchema(rdd, LabeledPoint.class); - schemaRDD.registerTempTable("points"); - List points = jsql.sql("SELECT label, features FROM points").collect(); - assert (points.size() == arr.size()); - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala deleted file mode 100644 index 94659ba95b1be..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/LabeledPointSuite.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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 - -import org.scalatest.FunSuite - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.SQLContext - -/** - * Test [[LabeledPoint]] - */ -class LabeledPointSuite extends FunSuite with MLlibTestSparkContext { - - @transient var sqlContext: SQLContext = _ - - override def beforeAll(): Unit = { - super.beforeAll() - sqlContext = new SQLContext(sc) - } - - test("LabeledPoint default weight 1.0") { - val label = 1.0 - val features = Vectors.dense(1.0, 2.0, 3.0) - val lp1 = LabeledPoint(label, features) - val lp2 = LabeledPoint(label, features, weight = 1.0) - assert(lp1 === lp2) - } - - test("Create SchemaRDD from RDD[LabeledPoint]") { - val sqlContext = this.sqlContext - import sqlContext._ - val arr = Seq( - LabeledPoint(0.0, Vectors.dense(1.0, 2.0, 3.0)), - LabeledPoint(1.0, Vectors.dense(1.1, 2.1, 3.1)), - LabeledPoint(0.0, Vectors.dense(1.2, 2.2, 3.2)), - LabeledPoint(1.0, Vectors.dense(1.3, 2.3, 3.3))) - val rdd = sc.parallelize(arr) - val schemaRDD = rdd.select('label, 'features) - val points = schemaRDD.collect() - assert(points.size === arr.size) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index df2374bf655e9..e7ee1a01e3796 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.classification import org.scalatest.FunSuite -import org.apache.spark.ml.LabeledPoint import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} @@ -45,13 +45,13 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(lr.getLabelCol == "label") val model = lr.fit(dataset) model.transform(dataset) - .select('label, 'score, 'prediction) + .select('label, 'probability, 'prediction) .collect() // Check defaults assert(model.getThreshold === 0.5) assert(model.getFeaturesCol == "features") assert(model.getPredictionCol == "prediction") - assert(model.getScoreCol == "score") + assert(model.getProbabilityCol == "probability") } test("logistic regression with setters") { @@ -60,7 +60,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { .setMaxIter(10) .setRegParam(1.0) .setThreshold(0.6) - .setScoreCol("probability") + .setProbabilityCol("myProbability") val model = lr.fit(dataset) model.transform(dataset, model.threshold -> 0.8) // overwrite threshold .select("label", "score", "prediction") @@ -80,26 +80,26 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { // Modify model params, and check that the params worked. model.setThreshold(1.0) val predAllZero = model.transform(dataset) - .select('prediction, 'probability) + .select('prediction, 'myProbability) .collect() - .map { case Row(pred: Double, prob: Double) => pred } + .map { case Row(pred: Double, prob: Vector) => pred } assert(predAllZero.forall(_ === 0.0)) // Call transform with params, and check that the params worked. val predNotAllZero = - model.transform(dataset, model.threshold -> 0.0, model.scoreCol -> "myProb") + model.transform(dataset, model.threshold -> 0.0, model.probabilityCol -> "myProb") .select('prediction, 'myProb) .collect() - .map { case Row(pred: Double, prob: Double) => pred } + .map { case Row(pred: Double, prob: Vector) => pred } assert(predNotAllZero.exists(_ !== 0.0)) // Call fit() with new params, and check as many params as we can. val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.threshold -> 0.4, - lr.scoreCol -> "theProb") + lr.probabilityCol -> "theProb") assert(model2.fittingParamMap.get(lr.maxIter).get === 5) assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) assert(model2.fittingParamMap.get(lr.threshold).get === 0.4) assert(model2.getThreshold === 0.4) - assert(model2.getScoreCol == "theProb") + assert(model2.getProbabilityCol == "theProb") } test("logistic regression: Predictor, Classifier methods") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index b126ccabfb37e..bfeae23268875 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.regression import org.scalatest.FunSuite -import org.apache.spark.ml.LabeledPoint import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{Row, SQLContext, SchemaRDD} From c3c8da507cbfd03ed5f2fc0fff6431fbf88862ce Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 30 Dec 2014 14:56:04 -0800 Subject: [PATCH 18/30] small cleanup --- .../apache/spark/ml/impl/estimator/Predictor.scala | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index b39875197f38d..f9f71a56ea39b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -80,23 +80,10 @@ abstract class Predictor[ M <: PredictionModel[FeaturesType, M]] extends Estimator[M] with PredictorParams { - // TODO: Eliminate asInstanceOf and see if that works. def setLabelCol(value: String): Learner = set(labelCol, value).asInstanceOf[Learner] def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] - /* - // This will be useful for boosting. - protected def selectLabelColumn(dataset: SchemaRDD, paramMap: ParamMap): RDD[Double] = { - import dataset.sqlContext._ - val map = this.paramMap ++ paramMap - dataset.select(map(labelCol).attr).map { - case Row(label: Double) => label - case Row(label: Int) => label.toDouble - } - } - */ - /** * :: DeveloperApi :: * From 0a16da9fdbd1133905e062748aa95eb43c96ddaf Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 30 Dec 2014 16:50:26 -0800 Subject: [PATCH 19/30] Fixed Linear/Logistic RegressionSuites --- .../LogisticRegressionSuite.scala | 50 +++++++------------ .../ml/regression/LinearRegressionSuite.scala | 23 --------- 2 files changed, 18 insertions(+), 55 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index e7ee1a01e3796..8cf7b81834918 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -21,7 +21,6 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} @@ -107,39 +106,26 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { import sqlContext._ val lr = new LogisticRegression - // fit() vs. train() - val model1 = lr.fit(dataset) - val rdd = dataset.select('label, 'features).map { case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } - val featuresRDD = rdd.map(_.features) - val model2 = lr.train(rdd) - assert(model1.intercept == model2.intercept) - assert(model1.weights.equals(model2.weights)) - assert(model1.numClasses == model2.numClasses) - assert(model1.numClasses === 2) - - // transform() vs. predict() - val trans = model1.transform(dataset).select('prediction) - val preds = model1.predict(rdd.map(_.features)) - trans.zip(preds).collect().foreach { case (Row(pred1: Double), pred2: Double) => - assert(pred1 == pred2) + val model = lr.fit(dataset) + assert(model.numClasses === 2) + + val threshold = model.getThreshold + val results = model.transform(dataset) + + // Compare rawPrediction with probability + results.select('rawPrediction, 'probability).collect().map { + case Row(raw: Vector, prob: Vector) => + val raw2prob: (Double => Double) = (m) => 1.0 / (1.0 + math.exp(-m)) + raw.toArray.map(raw2prob).zip(prob.toArray).foreach { case (r, p) => + assert(r ~== p relTol eps) + } } - // Check various types of predictions. - val rawPredictions = model1.predictRaw(featuresRDD) - val probabilities = model1.predictProbabilities(featuresRDD) - val predictions = model1.predict(featuresRDD) - val threshold = model1.getThreshold - rawPredictions.zip(probabilities).collect().foreach { case (raw: Vector, prob: Vector) => - val computeProbFromRaw: (Double => Double) = (m) => 1.0 / (1.0 + math.exp(-m)) - raw.toArray.map(computeProbFromRaw).zip(prob.toArray).foreach { case (r, p) => - assert(r ~== p relTol eps) - } - } - probabilities.zip(predictions).collect().foreach { case (prob: Vector, pred: Double) => - val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 - assert(pred == predFromProb) + // Compare prediction with probability + results.select('prediction, 'probability).collect().map { + case Row(pred: Double, prob: Vector) => + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index bfeae23268875..b5876cc96c2b8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -69,27 +69,4 @@ class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) assert(model2.getPredictionCol == "thePred") } - - test("linear regression: Predictor, Regressor methods") { - val sqlContext = this.sqlContext - import sqlContext._ - val lr = new LinearRegression - - // fit() vs. train() - val model1 = lr.fit(dataset) - val rdd = dataset.select('label, 'features).map { case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } - val features = rdd.map(_.features) - val model2 = lr.train(rdd) - assert(model1.intercept == model2.intercept) - assert(model1.weights.equals(model2.weights)) - - // transform() vs. predict() - val trans = model1.transform(dataset).select('prediction) - val preds = model1.predict(rdd.map(_.features)) - trans.zip(preds).collect().foreach { case (Row(pred1: Double), pred2: Double) => - assert(pred1 == pred2) - } - } } From 82f340bcbc0ec03127919239e32c30fef744ce75 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 30 Dec 2014 18:49:35 -0800 Subject: [PATCH 20/30] Fixed bug in LogisticRegression (introduced in this PR). Fixed Java suites --- .../classification/LogisticRegression.scala | 6 +- .../apache/spark/ml/JavaPipelineSuite.java | 2 +- .../JavaLinearRegressionSuite.java | 31 ++----- .../JavaLogisticRegressionSuite.java | 88 +++++++------------ .../LogisticRegressionSuite.scala | 13 +-- 5 files changed, 49 insertions(+), 91 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 2653af994d7c6..1cd9341598723 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -149,7 +149,7 @@ class LogisticRegressionModel private[ml] ( if (map(probabilityCol) != "") { if (map(rawPredictionCol) != "") { val raw2prob: Vector => Vector = (rawPreds) => { - val prob1 = 1.0 / 1.0 + math.exp(-rawPreds(1)) + val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) Vectors.dense(1.0 - prob1, prob1) } tmpData = tmpData.select(Star(None), @@ -171,7 +171,7 @@ class LogisticRegressionModel private[ml] ( predict.call(map(probabilityCol).attr) as map(predictionCol)) } else if (map(rawPredictionCol) != "") { val predict: Vector => Double = (rawPreds) => { - val prob1 = 1.0 / 1.0 + math.exp(-rawPreds(1)) + val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) if (prob1 > t) 1.0 else 0.0 } tmpData = tmpData.select(Star(None), @@ -207,7 +207,7 @@ class LogisticRegressionModel private[ml] ( override protected def predictRaw(features: Vector): Vector = { val m = margin(features) - Vectors.dense(-m, m) + Vectors.dense(0.0, m) } override protected def copy(): LogisticRegressionModel = { diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index 56a9dbdd58b64..50995ffef9ad5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -65,7 +65,7 @@ public void pipeline() { .setStages(new PipelineStage[] {scaler, lr}); PipelineModel model = pipeline.fit(dataset); model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java index 8662d68cd365b..1f47b711ac6d4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java @@ -30,12 +30,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import org.apache.spark.ml.LabeledPoint; import org.apache.spark.ml.regression.LinearRegression; import org.apache.spark.ml.regression.LinearRegressionModel; import static org.apache.spark.mllib.classification.LogisticRegressionSuite .generateLogisticInputAsList; import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.api.java.JavaSQLContext; import org.apache.spark.sql.api.java.JavaSchemaRDD; import org.apache.spark.sql.api.java.Row; @@ -93,35 +93,14 @@ public void linearRegressionWithSetters() { .setMaxIter(10) .setRegParam(1.0); LinearRegressionModel model = lr.fit(dataset); - assert(model.fittingParamMap().get(lr.maxIter()).get() == 10); - assert(model.fittingParamMap().get(lr.regParam()).get() == 1.0); + assert(model.fittingParamMap().apply(lr.maxIter()) == 10); + assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); // Call fit() with new params, and check as many params as we can. LinearRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); - assert(model2.fittingParamMap().get(lr.maxIter()).get() == 5); - assert(model2.fittingParamMap().get(lr.regParam()).get() == 0.1); + assert(model2.fittingParamMap().apply(lr.maxIter()) == 5); + assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); assert(model2.getPredictionCol().equals("thePred")); } - - @Test - public void linearRegressionPredictorClassifierMethods() { - LinearRegression lr = new LinearRegression(); - - // fit() vs. train() - LinearRegressionModel model1 = lr.fit(dataset); - LinearRegressionModel model2 = lr.train(datasetRDD); - assert(model1.intercept() == model2.intercept()); - assert(model1.weights().equals(model2.weights())); - - // transform() vs. predict() - model1.transform(dataset).registerTempTable("transformed"); - JavaSchemaRDD trans = jsql.sql("SELECT prediction FROM transformed"); - JavaRDD preds = model1.predict(featuresRDD); - for (Tuple2 trans_pred: trans.zip(preds).collect()) { - double t = trans_pred._1().getDouble(0); - double p = trans_pred._2(); - assert(t == p); - } - } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 3ad15e516c16e..11acaa3a0d357 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -17,8 +17,6 @@ package org.apache.spark.ml.classification; -import scala.Tuple2; - import java.io.Serializable; import java.lang.Math; import java.util.ArrayList; @@ -34,9 +32,8 @@ import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.SQLContext; import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; -import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.ml.LabeledPoint; +import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Row; @@ -47,7 +44,6 @@ public class JavaLogisticRegressionSuite implements Serializable { private transient DataFrame dataset; private transient JavaRDD datasetRDD; - private transient JavaRDD featuresRDD; private double eps = 1e-5; @Before @@ -60,9 +56,6 @@ public void setUp() { points.add(new LabeledPoint(lp.label(), lp.features())); } datasetRDD = jsc.parallelize(points, 2); - featuresRDD = datasetRDD.map(new Function() { - @Override public Vector call(LabeledPoint lp) { return lp.features(); } - }); dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); dataset.registerTempTable("dataset"); } @@ -79,13 +72,13 @@ public void logisticRegressionDefaultParams() { assert(lr.getLabelCol().equals("label")); LogisticRegressionModel model = lr.fit(dataset); model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); // Check defaults assert(model.getThreshold() == 0.5); assert(model.getFeaturesCol().equals("features")); assert(model.getPredictionCol().equals("prediction")); - assert(model.getScoreCol().equals("score")); + assert(model.getProbabilityCol().equals("probability")); } @Test @@ -95,17 +88,17 @@ public void logisticRegressionWithSetters() { .setMaxIter(10) .setRegParam(1.0) .setThreshold(0.6) - .setScoreCol("probability"); + .setProbabilityCol("myProbability"); LogisticRegressionModel model = lr.fit(dataset); - assert(model.fittingParamMap().get(lr.maxIter()).get() == 10); - assert(model.fittingParamMap().get(lr.regParam()).get() == 1.0); - assert(model.fittingParamMap().get(lr.threshold()).get() == 0.6); + assert(model.fittingParamMap().apply(lr.maxIter()) == 10); + assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); + assert(model.fittingParamMap().apply(lr.threshold()).equals(0.6)); assert(model.getThreshold() == 0.6); // Modify model params, and check that the params worked. model.setThreshold(1.0); model.transform(dataset).registerTempTable("predAllZero"); - SchemaRDD predAllZero = jsql.sql("SELECT prediction, probability FROM predAllZero"); + SchemaRDD predAllZero = jsql.sql("SELECT prediction, myProbability FROM predAllZero"); for (Row r: predAllZero.collectAsList()) { assert(r.getDouble(0) == 0.0); } @@ -117,7 +110,7 @@ public void logisticRegressionWithSetters() { predictions.collectAsList(); */ - model.transform(dataset, model.threshold().w(0.0), model.scoreCol().w("myProb")) + model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) .registerTempTable("predNotAllZero"); SchemaRDD predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); boolean foundNonZero = false; @@ -128,54 +121,37 @@ public void logisticRegressionWithSetters() { // Call fit() with new params, and check as many params as we can. LogisticRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), - lr.threshold().w(0.4), lr.scoreCol().w("theProb")); - assert(model2.fittingParamMap().get(lr.maxIter()).get() == 5); - assert(model2.fittingParamMap().get(lr.regParam()).get() == 0.1); - assert(model2.fittingParamMap().get(lr.threshold()).get() == 0.4); + lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); + assert(model2.fittingParamMap().apply(lr.maxIter()) == 5); + assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); + assert(model2.fittingParamMap().apply(lr.threshold()).equals(0.4)); assert(model2.getThreshold() == 0.4); - assert(model2.getScoreCol().equals("theProb")); + assert(model2.getProbabilityCol().equals("theProb")); } + @SuppressWarnings("unchecked") @Test public void logisticRegressionPredictorClassifierMethods() { LogisticRegression lr = new LogisticRegression(); - - // fit() vs. train() - LogisticRegressionModel model1 = lr.fit(dataset); - LogisticRegressionModel model2 = lr.train(datasetRDD); - assert(model1.intercept() == model2.intercept()); - assert(model1.weights().equals(model2.weights())); - assert(model1.numClasses() == model2.numClasses()); - assert(model1.numClasses() == 2); - - // transform() vs. predict() - model1.transform(dataset).registerTempTable("transformed"); - SchemaRDD trans = jsql.sql("SELECT prediction FROM transformed"); - JavaRDD preds = model1.predict(featuresRDD); - for (scala.Tuple2 trans_pred: trans.toJavaRDD().zip(preds).collect()) { - double t = trans_pred._1().getDouble(0); - double p = trans_pred._2(); - assert(t == p); + LogisticRegressionModel model = lr.fit(dataset); + assert(model.numClasses() == 2); + + model.transform(dataset).registerTempTable("transformed"); + SchemaRDD trans1 = jsql.sql("SELECT rawPrediction, probability FROM transformed"); + for (Row row: trans1.collect()) { + Vector raw = (Vector)row.get(0); + Vector prob = (Vector)row.get(1); + assert(raw.size() == 2); + assert(prob.size() == 2); + double probFromRaw1 = 1.0 / (1.0 + Math.exp(-raw.apply(1))); + assert(Math.abs(prob.apply(1) - probFromRaw1) < eps); + assert(Math.abs(prob.apply(0) - (1.0 - probFromRaw1)) < eps); } - // Check various types of predictions. - JavaRDD rawPredictions = model1.predictRaw(featuresRDD); - JavaRDD probabilities = model1.predictProbabilities(featuresRDD); - JavaRDD predictions = model1.predict(featuresRDD); - double threshold = model1.getThreshold(); - for (Tuple2 raw_prob: rawPredictions.zip(probabilities).collect()) { - Vector raw = raw_prob._1(); - Vector prob = raw_prob._2(); - for (int i = 0; i < raw.size(); ++i) { - double r = raw.apply(i); - double p = prob.apply(i); - double pFromR = 1.0 / (1.0 + Math.exp(-r)); - assert(Math.abs(r - pFromR) < eps); - } - } - for (Tuple2 prob_pred: probabilities.zip(predictions).collect()) { - Vector prob = prob_pred._1(); - double pred = prob_pred._2(); + SchemaRDD trans2 = jsql.sql("SELECT prediction, probability FROM transformed"); + for (Row row: trans2.collect()) { + double pred = row.getDouble(0); + Vector prob = (Vector)row.get(1); double probOfPred = prob.apply((int)pred); for (int i = 0; i < prob.size(); ++i) { assert(probOfPred >= prob.apply(i)); diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 8cf7b81834918..f412622572c1b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -82,7 +82,9 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { .select('prediction, 'myProbability) .collect() .map { case Row(pred: Double, prob: Vector) => pred } - assert(predAllZero.forall(_ === 0.0)) + assert(predAllZero.forall(_ === 0), + s"With threshold=1.0, expected predictions to be all 0, but only" + + s" ${predAllZero.count(_ === 0)} of ${dataset.count()} were 0.") // Call transform with params, and check that the params worked. val predNotAllZero = model.transform(dataset, model.threshold -> 0.0, model.probabilityCol -> "myProb") @@ -115,10 +117,11 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { // Compare rawPrediction with probability results.select('rawPrediction, 'probability).collect().map { case Row(raw: Vector, prob: Vector) => - val raw2prob: (Double => Double) = (m) => 1.0 / (1.0 + math.exp(-m)) - raw.toArray.map(raw2prob).zip(prob.toArray).foreach { case (r, p) => - assert(r ~== p relTol eps) - } + assert(raw.size === 2) + assert(prob.size === 2) + val probFromRaw1 = 1.0 / (1.0 + math.exp(-raw(1))) + assert(prob(1) ~== probFromRaw1 relTol eps) + assert(prob(0) ~== 1.0 - probFromRaw1 relTol eps) } // Compare prediction with probability From 343e7bdd1ce2b676d130cca1116eb1bf1c5b5d0e Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 30 Dec 2014 23:40:19 -0800 Subject: [PATCH 21/30] added blanket mima exclude for ml package --- project/MimaExcludes.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b17532c1d814c..b9df8dcb7b953 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,7 @@ object MimaExcludes { case v if v.startsWith("1.3") => Seq( MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("ml"), // These are needed if checking against the sbt build, since they are part of // the maven-generated artifacts in the 1.2 build. MimaBuild.excludeSparkPackage("unused"), From f549e34a415577b104c7a187df05a6f9147f88da Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 14 Jan 2015 19:26:02 -0800 Subject: [PATCH 22/30] Updates based on code review. Major ones are: * Created weakly typed Predictor.train() method which is called by fit() so that developers do not have to call schema validation or copy parameters. * Made Predictor.featuresDataType have a default value of VectorUDT. * NOTE: This could be dangerous since the FeaturesType type parameter cannot have a default value. --- .../examples/ml/CrossValidatorExample.scala | 2 +- .../examples/ml/DeveloperApiExample.scala | 50 +++++++------------ .../examples/ml/SimpleParamsExample.scala | 2 +- .../ml/SimpleTextClassificationPipeline.scala | 2 +- .../classification/LogisticRegression.scala | 21 ++------ .../spark/ml/impl/estimator/Predictor.scala | 35 +++++++++++-- .../apache/spark/ml/param/sharedParams.scala | 4 +- .../ml/regression/LinearRegression.scala | 21 ++------ 8 files changed, 65 insertions(+), 72 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index 20fcf132be15b..0aadd476cba63 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -104,7 +104,7 @@ object CrossValidatorExample { .select('id, 'text, 'probability, 'prediction) .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println("(" + id + ", " + text + ") --> prob=" + prob + ", prediction=" + prediction) + println(s"($id, $text) --> prob=$prob, prediction=$prediction") } sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index deffce192b2b4..002641798b0c6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -21,9 +21,9 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.ml.classification.{Classifier, ClassifierParams, ClassificationModel} import org.apache.spark.ml.param.{Params, IntParam, ParamMap} -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{DataType, SchemaRDD, Row, SQLContext} +import org.apache.spark.sql.{SchemaRDD, Row, SQLContext} /** * A simple example demonstrating how to write your own learning algorithm using Estimator, @@ -85,7 +85,14 @@ object DeveloperApiExample { */ private trait MyLogisticRegressionParams extends ClassifierParams { - /** param for max number of iterations */ + /** + * Param for max number of iterations + * + * NOTE: The usual way to add a parameter to a model or algorithm is to include: + * - val myParamName: ParamType + * - def getMyParamName + * - def setMyParamName + */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") def getMaxIter: Int = get(maxIter) } @@ -101,40 +108,23 @@ private class MyLogisticRegression setMaxIter(100) // Initialize + // The parameter setter is in this class since it should return type MyLogisticRegression. def setMaxIter(value: Int): this.type = set(maxIter, value) - override def fit(dataset: SchemaRDD, paramMap: ParamMap): MyLogisticRegressionModel = { - // Check schema (types). This allows early failure before running the algorithm. - transformSchema(dataset.schema, paramMap, logging = true) - + // This method is used by fit() + override protected def train( + dataset: SchemaRDD, + paramMap: ParamMap): MyLogisticRegressionModel = { // Extract columns from data using helper method. val oldDataset = extractLabeledPoints(dataset, paramMap) - // Combine given parameters with the embedded parameters, where the given paramMap overrides - // any embedded settings. - val map = this.paramMap ++ paramMap - // Do learning to estimate the weight vector. val numFeatures = oldDataset.take(1)(0).features.size val weights = Vectors.zeros(numFeatures) // Learning would happen here. - // Create a model to return. - val lrm = new MyLogisticRegressionModel(this, map, weights) - - // Copy model params. - // An Estimator stores the parameters for the Model it produces, and this copies any relevant - // parameters to the model. - Params.inheritValues(map, this, lrm) - - // Return the learned model. - lrm + // Create a model, and return it. + new MyLogisticRegressionModel(this, paramMap, weights) } - - /** - * Returns the SQL DataType corresponding to the FeaturesType type parameter. - * This is used by [[ClassifierParams.validateAndTransformSchema()]] to check the input data. - */ - override protected def featuresDataType: DataType = new VectorUDT } /** @@ -186,10 +176,4 @@ private class MyLogisticRegressionModel( Params.inheritValues(this.paramMap, this, m) m } - - /** - * Returns the SQL DataType corresponding to the FeaturesType type parameter. - * This is used by [[ClassifierParams.validateAndTransformSchema()]] to check the input data. - */ - override protected def featuresDataType: DataType = new VectorUDT } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 80d130728c85f..ed969f6b64fdc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -94,7 +94,7 @@ object SimpleParamsExample { .select('features, 'label, 'myProbability, 'prediction) .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => - println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) + println("($features, $label) -> prob=$prob, prediction=$prediction") } sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 0a5adaa7fc1ed..ab93c4847195e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -83,7 +83,7 @@ object SimpleTextClassificationPipeline { .select('id, 'text, 'probability, 'prediction) .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println("(" + id + ", " + text + ") --> prob=" + prob + ", prediction=" + prediction) + println("($id, $text) --> prob=$prob, prediction=$prediction") } sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 1cd9341598723..4b7aa6ece7130 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT, Vectors} +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.sql._ import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -52,13 +52,9 @@ class LogisticRegression def setMaxIter(value: Int): this.type = set(maxIter, value) def setThreshold(value: Double): this.type = set(threshold, value) - override def fit(dataset: SchemaRDD, paramMap: ParamMap): LogisticRegressionModel = { - // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - + override protected def train(dataset: SchemaRDD, paramMap: ParamMap): LogisticRegressionModel = { // Extract columns from data. If dataset is persisted, do not persist oldDataset. val oldDataset = extractLabeledPoints(dataset, paramMap) - val map = this.paramMap ++ paramMap val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) @@ -67,21 +63,16 @@ class LogisticRegression // Train model val lr = new LogisticRegressionWithLBFGS lr.optimizer - .setRegParam(map(regParam)) - .setNumIterations(map(maxIter)) + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) val oldModel = lr.run(oldDataset) - val lrm = new LogisticRegressionModel(this, map, oldModel.weights, oldModel.intercept) + val lrm = new LogisticRegressionModel(this, paramMap, oldModel.weights, oldModel.intercept) if (handlePersistence) { oldDataset.unpersist() } - - // copy model params - Params.inheritValues(map, this, lrm) lrm } - - override protected def featuresDataType: DataType = new VectorUDT } @@ -215,6 +206,4 @@ class LogisticRegressionModel private[ml] ( Params.inheritValues(this.paramMap, this, m) m } - - override protected def featuresDataType: DataType = new VectorUDT } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index f9f71a56ea39b..4a166c9c87321 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.impl.estimator import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{VectorUDT, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -84,6 +84,31 @@ abstract class Predictor[ def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] + override def fit(dataset: SchemaRDD, paramMap: ParamMap): M = { + // This handles a few items such as schema validation. + // Developers only need to implement train(). + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + val model = train(dataset, map) + Params.inheritValues(map, this, model) // copy params to model + model + } + + /** + * :: DeveloperApi :: + * + * Train a model using the given dataset and parameters. + * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation + * and copying parameters into the model. + * + * @param dataset Training dataset + * @param paramMap Parameter map. Unlike [[fit()]]'s paramMap, this paramMap has already + * been combined with the embedded ParamMap. + * @return Fitted model + */ + @DeveloperApi + protected def train(dataset: SchemaRDD, paramMap: ParamMap): M + /** * :: DeveloperApi :: * @@ -91,9 +116,11 @@ abstract class Predictor[ * * This is used by [[validateAndTransformSchema()]]. * This workaround is needed since SQL has different APIs for Scala and Java. + * + * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ @DeveloperApi - protected def featuresDataType: DataType + protected def featuresDataType: DataType = new VectorUDT private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { validateAndTransformSchema(schema, paramMap, fitting = true, featuresDataType) @@ -138,9 +165,11 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, * * This is used by [[validateAndTransformSchema()]]. * This workaround is needed since SQL has different APIs for Scala and Java. + * + * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ @DeveloperApi - protected def featuresDataType: DataType + protected def featuresDataType: DataType = new VectorUDT private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { validateAndTransformSchema(schema, paramMap, fitting = false, featuresDataType) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index bf336f3f7173b..32fc74462ef4a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -18,7 +18,9 @@ package org.apache.spark.ml.param /* NOTE TO DEVELOPERS: - * If you add these parameter traits into your algorithm, you need to add a setter method as well. + * If you mix these parameter traits into your algorithm, please add a setter method as well + * so that users may use a builder pattern: + * val myLearner = new MyLearner().setParam1(x).setParam2(y)... */ private[ml] trait HasRegParam extends Params { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 8ac2738bfe5b5..72f8266018bc0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} -import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector} +import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.sql._ import org.apache.spark.storage.StorageLevel @@ -45,13 +45,9 @@ class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegress def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) - override def fit(dataset: SchemaRDD, paramMap: ParamMap): LinearRegressionModel = { - // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - + override protected def train(dataset: SchemaRDD, paramMap: ParamMap): LinearRegressionModel = { // Extract columns from data. If dataset is persisted, do not persist oldDataset. val oldDataset = extractLabeledPoints(dataset, paramMap) - val map = this.paramMap ++ paramMap val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) @@ -60,21 +56,16 @@ class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegress // Train model val lr = new LinearRegressionWithSGD() lr.optimizer - .setRegParam(map(regParam)) - .setNumIterations(map(maxIter)) + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) val model = lr.run(oldDataset) - val lrm = new LinearRegressionModel(this, map, model.weights, model.intercept) + val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) if (handlePersistence) { oldDataset.unpersist() } - - // copy model params - Params.inheritValues(map, this, lrm) lrm } - - override protected def featuresDataType: DataType = new VectorUDT } /** @@ -100,6 +91,4 @@ class LinearRegressionModel private[ml] ( Params.inheritValues(this.paramMap, this, m) m } - - override protected def featuresDataType: DataType = new VectorUDT } From 216d19919a66602c62b21106ae08bedff5bbe42c Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 15 Jan 2015 11:57:08 -0800 Subject: [PATCH 23/30] fixed after sql datatypes PR got merged --- .../org/apache/spark/examples/ml/DeveloperApiExample.scala | 2 +- .../scala/org/apache/spark/ml/classification/Classifier.scala | 2 ++ .../apache/spark/ml/classification/LogisticRegression.scala | 3 +++ .../spark/ml/classification/ProbabilisticClassifier.scala | 2 ++ .../spark/ml/evaluation/BinaryClassificationEvaluator.scala | 1 + .../scala/org/apache/spark/ml/impl/estimator/Predictor.scala | 1 + mllib/src/main/scala/org/apache/spark/ml/param/params.scala | 3 ++- .../org/apache/spark/ml/regression/LinearRegression.scala | 2 ++ 8 files changed, 14 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 002641798b0c6..13a860a86afb0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -18,13 +18,13 @@ package org.apache.spark.examples.ml import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.ml.classification.{Classifier, ClassifierParams, ClassificationModel} import org.apache.spark.ml.param.{Params, IntParam, ParamMap} import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.{SchemaRDD, Row, SQLContext} + /** * A simple example demonstrating how to write your own learning algorithm using Estimator, * Transformer, and other abstractions. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 34e4b6fe298d1..58a48bc5ecce3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -23,6 +23,8 @@ import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.types.{DataType, StructType} + /** * :: DeveloperApi :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 4b7aa6ece7130..7dcc6b1bf9ecd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -24,8 +24,11 @@ import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.sql._ import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} +import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.catalyst.dsl._ import org.apache.spark.storage.StorageLevel + /** * Params for logistic regression. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index b4da63111f830..d7c75ef6e364b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -22,6 +22,8 @@ import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.types.{DataType, StructType} + /** * Params for probabilistic classification. diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index af141e95ee9e0..aa1caff0c0931 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.types.DoubleType + /** * :: AlphaComponent :: * diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 4a166c9c87321..5e6a8912dabb8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -25,6 +25,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 0358309991655..c4f98a7222d06 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -24,7 +24,8 @@ import java.lang.reflect.Modifier import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.Identifiable -import org.apache.spark.sql._ +import org.apache.spark.sql.types.{DataType, StructField, StructType} + /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 72f8266018bc0..29080cb0100fd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -24,12 +24,14 @@ import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.sql._ import org.apache.spark.storage.StorageLevel + /** * Params for linear regression. */ private[regression] trait LinearRegressionParams extends RegressorParams with HasRegParam with HasMaxIter + /** * :: AlphaComponent :: * From f5429971dcb067628780332723345d7c9da30865 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 19 Jan 2015 14:12:36 -0800 Subject: [PATCH 24/30] Added MIMA excludes for VectorUDT (now public), and added DeveloperApi annotation to it --- .../main/scala/org/apache/spark/mllib/linalg/Vectors.scala | 4 ++++ project/MimaExcludes.scala | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 66ee01cdc0d34..6a5259d848249 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.util.NumericParser import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow @@ -110,9 +111,12 @@ sealed trait Vector extends Serializable { } /** + * :: DeveloperApi :: + * * User-defined type for [[Vector]] which allows easy interaction with SQL * via [[org.apache.spark.sql.DataFrame]]. */ +@DeveloperApi class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b9df8dcb7b953..4065a562a1a18 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -143,6 +143,11 @@ object MimaExcludes { "org.apache.spark.graphx.Graph.getCheckpointFiles"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.graphx.Graph.isCheckpointed") + ) ++ Seq( + // SPARK-4789 Standardize ML Prediction APIs + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.linalg.VectorUDT"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.sqlType") ) case v if v.startsWith("1.2") => From 9872424607a39699a5b23cc83b3ebfd7be062957 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 19 Jan 2015 15:53:15 -0800 Subject: [PATCH 25/30] fixed JavaLinearRegressionSuite.java Java sql api --- .../JavaLinearRegressionSuite.java | 21 ++++++------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java index 1f47b711ac6d4..d918fc7caf6a0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java @@ -17,8 +17,6 @@ package org.apache.spark.ml.classification; -import scala.Tuple2; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; @@ -29,40 +27,33 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.regression.LinearRegression; import org.apache.spark.ml.regression.LinearRegressionModel; import static org.apache.spark.mllib.classification.LogisticRegressionSuite .generateLogisticInputAsList; -import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SchemaRDD; public class JavaLinearRegressionSuite implements Serializable { private transient JavaSparkContext jsc; - private transient JavaSQLContext jsql; - private transient JavaSchemaRDD dataset; + private transient SQLContext jsql; + private transient SchemaRDD dataset; private transient JavaRDD datasetRDD; - private transient JavaRDD featuresRDD; private double eps = 1e-5; @Before public void setUp() { jsc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - jsql = new JavaSQLContext(jsc); + jsql = new SQLContext(jsc); List points = new ArrayList(); for (org.apache.spark.mllib.regression.LabeledPoint lp: generateLogisticInputAsList(1.0, 1.0, 100, 42)) { points.add(new LabeledPoint(lp.label(), lp.features())); } datasetRDD = jsc.parallelize(points, 2); - featuresRDD = datasetRDD.map(new Function() { - @Override public Vector call(LabeledPoint lp) { return lp.features(); } - }); dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); dataset.registerTempTable("dataset"); } @@ -79,7 +70,7 @@ public void linearRegressionDefaultParams() { assert(lr.getLabelCol().equals("label")); LinearRegressionModel model = lr.fit(dataset); model.transform(dataset).registerTempTable("prediction"); - JavaSchemaRDD predictions = jsql.sql("SELECT label, prediction FROM prediction"); + SchemaRDD predictions = jsql.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults assert(model.getFeaturesCol().equals("features")); From bcb9549f57d9f8cff5ee534e897bb2772cc966bf Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 29 Jan 2015 18:26:37 -0800 Subject: [PATCH 26/30] Fixed issues after rebasing from master (after move from SchemaRDD to DataFrame) --- .../spark/ml/classification/Classifier.scala | 30 +++++------ .../classification/LogisticRegression.scala | 51 +++++++------------ .../ProbabilisticClassifier.scala | 13 +++-- .../BinaryClassificationEvaluator.scala | 6 +-- .../spark/ml/impl/estimator/Predictor.scala | 20 ++++---- .../ml/regression/LinearRegression.scala | 6 +-- .../JavaLogisticRegressionSuite.java | 27 +++------- .../JavaLinearRegressionSuite.java | 18 ++----- .../LogisticRegressionSuite.scala | 27 ++++------ .../ml/regression/LinearRegressionSuite.scala | 15 ++---- 10 files changed, 79 insertions(+), 134 deletions(-) rename mllib/src/test/java/org/apache/spark/ml/{classification => regression}/JavaLinearRegressionSuite.java (83%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 58a48bc5ecce3..ca791ae612132 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -21,9 +21,9 @@ import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.Star -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** @@ -95,7 +95,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset */ - override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { // This default implementation should be overridden as needed. // Check schema @@ -162,12 +162,9 @@ private[ml] object ClassificationModel { * @return (number of columns added, transformed dataset) */ private[ml] def transformColumnsImpl[FeaturesType]( - dataset: SchemaRDD, + dataset: DataFrame, model: ClassificationModel[FeaturesType, _], - map: ParamMap): (Int, SchemaRDD) = { - - import org.apache.spark.sql.catalyst.dsl._ - import dataset.sqlContext._ + map: ParamMap): (Int, DataFrame) = { // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. @@ -176,22 +173,25 @@ private[ml] object ClassificationModel { if (map(model.rawPredictionCol) != "") { // output raw prediction val features2raw: FeaturesType => Vector = model.predictRaw - tmpData = tmpData.select(Star(None), - features2raw.call(map(model.featuresCol).attr) as map(model.rawPredictionCol)) + tmpData = tmpData.select($"*", + callUDF(features2raw, new VectorUDT, + tmpData(map(model.featuresCol))).as(map(model.rawPredictionCol))) numColsOutput += 1 if (map(model.predictionCol) != "") { val raw2pred: Vector => Double = (rawPred) => { rawPred.toArray.zipWithIndex.maxBy(_._1)._2 } - tmpData = tmpData.select(Star(None), - raw2pred.call(map(model.rawPredictionCol).attr) as map(model.predictionCol)) + tmpData = tmpData.select($"*", + callUDF(raw2pred, DoubleType, + tmpData(map(model.rawPredictionCol))).as(map(model.predictionCol))) numColsOutput += 1 } } else if (map(model.predictionCol) != "") { // output prediction val features2pred: FeaturesType => Double = model.predict - tmpData = tmpData.select(Star(None), - features2pred.call(map(model.featuresCol).attr) as map(model.predictionCol)) + tmpData = tmpData.select($"*", + callUDF(features2pred, DoubleType, + tmpData(map(model.featuresCol))).as(map(model.predictionCol))) numColsOutput += 1 } (numColsOutput, tmpData) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 7dcc6b1bf9ecd..293062fc45915 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -20,12 +20,10 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} -import org.apache.spark.sql._ +import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Dsl._ -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} -import org.apache.spark.sql.catalyst.analysis.Star -import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -55,10 +53,10 @@ class LogisticRegression def setMaxIter(value: Int): this.type = set(maxIter, value) def setThreshold(value: Double): this.type = set(threshold, value) - override protected def train(dataset: SchemaRDD, paramMap: ParamMap): LogisticRegressionModel = { + override protected def train(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = { // Extract columns from data. If dataset is persisted, do not persist oldDataset. val oldDataset = extractLabeledPoints(dataset, paramMap) - val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) } @@ -106,25 +104,10 @@ class LogisticRegressionModel private[ml] ( 1.0 / (1.0 + math.exp(-m)) } -/* override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val scoreFunction = udf { v: Vector => - val margin = BLAS.dot(v, weights) - 1.0 / (1.0 + math.exp(-margin)) - } - val t = map(threshold) - val predictFunction: Double => Double = (score) => { if (score > t) 1.0 else 0.0 } - dataset - .select($"*", callUDF(scoreFunction, col(map(featuresCol))).as(map(scoreCol))) - .select($"*", callUDF(predictFunction, col(map(scoreCol))).as(map(predictionCol))) -*/ - override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { // Check schema transformSchema(dataset.schema, paramMap, logging = true) - import dataset.sqlContext._ val map = this.paramMap ++ paramMap // Output selected columns only. @@ -136,8 +119,8 @@ class LogisticRegressionModel private[ml] ( var numColsOutput = 0 if (map(rawPredictionCol) != "") { val features2raw: Vector => Vector = predictRaw - tmpData = tmpData.select(Star(None), - features2raw.call(map(featuresCol).attr) as map(rawPredictionCol)) + tmpData = tmpData.select($"*", + callUDF(features2raw, new VectorUDT, tmpData(map(featuresCol))).as(map(rawPredictionCol))) numColsOutput += 1 } if (map(probabilityCol) != "") { @@ -146,12 +129,12 @@ class LogisticRegressionModel private[ml] ( val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) Vectors.dense(1.0 - prob1, prob1) } - tmpData = tmpData.select(Star(None), - raw2prob.call(map(rawPredictionCol).attr) as map(probabilityCol)) + tmpData = tmpData.select($"*", + callUDF(raw2prob, new VectorUDT, tmpData(map(rawPredictionCol))).as(map(probabilityCol))) } else { val features2prob: Vector => Vector = predictProbabilities - tmpData = tmpData.select(Star(None), - features2prob.call(map(featuresCol).attr) as map(probabilityCol)) + tmpData = tmpData.select($"*", + callUDF(features2prob, new VectorUDT, tmpData(map(featuresCol))).as(map(probabilityCol))) } numColsOutput += 1 } @@ -161,19 +144,19 @@ class LogisticRegressionModel private[ml] ( val predict: Vector => Double = (probs) => { if (probs(1) > t) 1.0 else 0.0 } - tmpData = tmpData.select(Star(None), - predict.call(map(probabilityCol).attr) as map(predictionCol)) + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, tmpData(map(probabilityCol))).as(map(predictionCol))) } else if (map(rawPredictionCol) != "") { val predict: Vector => Double = (rawPreds) => { val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) if (prob1 > t) 1.0 else 0.0 } - tmpData = tmpData.select(Star(None), - predict.call(map(rawPredictionCol).attr) as map(predictionCol)) + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, tmpData(map(rawPredictionCol))).as(map(predictionCol))) } else { val predict: Vector => Double = this.predict - tmpData = tmpData.select(Star(None), - predict.call(map(featuresCol).attr) as map(predictionCol)) + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, tmpData(map(featuresCol))).as(map(predictionCol))) } numColsOutput += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index d7c75ef6e364b..330571cf7b91f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -20,8 +20,8 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DataType, StructType} @@ -91,10 +91,8 @@ abstract class ProbabilisticClassificationModel[ * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset */ - override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { // This default implementation should be overridden as needed. - import dataset.sqlContext._ - import org.apache.spark.sql.catalyst.dsl._ // Check schema transformSchema(dataset.schema, paramMap, logging = true) @@ -118,8 +116,9 @@ abstract class ProbabilisticClassificationModel[ val features2probs: FeaturesType => Vector = (features) => { tmpModel.predictProbabilities(features) } - outputData.select(Star(None), - features2probs.call(map(featuresCol).attr) as map(probabilityCol)) + outputData.select($"*", + callUDF(features2probs, new VectorUDT, + outputData(map(featuresCol))).as(map(probabilityCol))) } else { if (numColsOutput == 0) { this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index aa1caff0c0931..f21a30627e540 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml._ +import org.apache.spark.ml.Evaluator import org.apache.spark.ml.param._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType @@ -52,7 +52,7 @@ class BinaryClassificationEvaluator extends Evaluator with Params checkInputColumn(schema, map(labelCol), DoubleType) // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. - val scoreAndLabels = dataset.select(map(rawPredictionCol).attr, map(labelCol).attr) + val scoreAndLabels = dataset.select(map(rawPredictionCol), map(labelCol)) .map { case Row(rawPrediction: Vector, label: Double) => (rawPrediction(1), label) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 5e6a8912dabb8..cdf458f584ae4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -23,8 +23,8 @@ import org.apache.spark.ml.param._ import org.apache.spark.mllib.linalg.{VectorUDT, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -85,7 +85,7 @@ abstract class Predictor[ def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] - override def fit(dataset: SchemaRDD, paramMap: ParamMap): M = { + override def fit(dataset: DataFrame, paramMap: ParamMap): M = { // This handles a few items such as schema validation. // Developers only need to implement train(). transformSchema(dataset.schema, paramMap, logging = true) @@ -108,7 +108,7 @@ abstract class Predictor[ * @return Fitted model */ @DeveloperApi - protected def train(dataset: SchemaRDD, paramMap: ParamMap): M + protected def train(dataset: DataFrame, paramMap: ParamMap): M /** * :: DeveloperApi :: @@ -131,10 +131,9 @@ abstract class Predictor[ * Extract [[labelCol]] and [[featuresCol]] from the given dataset, * and put it in an RDD with strong types. */ - protected def extractLabeledPoints(dataset: SchemaRDD, paramMap: ParamMap): RDD[LabeledPoint] = { - import dataset.sqlContext._ + protected def extractLabeledPoints(dataset: DataFrame, paramMap: ParamMap): RDD[LabeledPoint] = { val map = this.paramMap ++ paramMap - dataset.select(map(labelCol).attr, map(featuresCol).attr) + dataset.select(map(labelCol), map(featuresCol)) .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } @@ -184,10 +183,8 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset with [[predictionCol]] of type [[Double]] */ - override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { // This default implementation should be overridden as needed. - import org.apache.spark.sql.catalyst.dsl._ - import dataset.sqlContext._ // Check schema transformSchema(dataset.schema, paramMap, logging = true) @@ -206,7 +203,8 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, val pred: FeaturesType => Double = (features) => { tmpModel.predict(features) } - dataset.select(Star(None), pred.call(map(featuresCol).attr) as map(predictionCol)) + dataset.select($"*", + callUDF(pred, DoubleType, dataset(map(featuresCol))).as(map(predictionCol))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + " since no output columns were set.") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 29080cb0100fd..d5a7bdafcb623 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.sql._ +import org.apache.spark.sql.DataFrame import org.apache.spark.storage.StorageLevel @@ -47,10 +47,10 @@ class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegress def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) - override protected def train(dataset: SchemaRDD, paramMap: ParamMap): LinearRegressionModel = { + override protected def train(dataset: DataFrame, paramMap: ParamMap): LinearRegressionModel = { // Extract columns from data. If dataset is persisted, do not persist oldDataset. val oldDataset = extractLabeledPoints(dataset, paramMap) - val handlePersistence = dataset.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 11acaa3a0d357..26284023b0f69 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -19,7 +19,6 @@ import java.io.Serializable; import java.lang.Math; -import java.util.ArrayList; import java.util.List; import org.junit.After; @@ -28,12 +27,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.Row; @@ -50,11 +48,7 @@ public class JavaLogisticRegressionSuite implements Serializable { public void setUp() { jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); jsql = new SQLContext(jsc); - List points = new ArrayList(); - for (org.apache.spark.mllib.regression.LabeledPoint lp: - generateLogisticInputAsList(1.0, 1.0, 100, 42)) { - points.add(new LabeledPoint(lp.label(), lp.features())); - } + List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); dataset.registerTempTable("dataset"); @@ -98,21 +92,14 @@ public void logisticRegressionWithSetters() { // Modify model params, and check that the params worked. model.setThreshold(1.0); model.transform(dataset).registerTempTable("predAllZero"); - SchemaRDD predAllZero = jsql.sql("SELECT prediction, myProbability FROM predAllZero"); + DataFrame predAllZero = jsql.sql("SELECT prediction, myProbability FROM predAllZero"); for (Row r: predAllZero.collectAsList()) { assert(r.getDouble(0) == 0.0); } // Call transform with params, and check that the params worked. - /* TODO: USE THIS - model.transform(dataset, model.threshold().w(0.8)) // overwrite threshold - .registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); - predictions.collectAsList(); - */ - model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) .registerTempTable("predNotAllZero"); - SchemaRDD predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); + DataFrame predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); boolean foundNonZero = false; for (Row r: predNotAllZero.collectAsList()) { if (r.getDouble(0) != 0.0) foundNonZero = true; @@ -137,7 +124,7 @@ public void logisticRegressionPredictorClassifierMethods() { assert(model.numClasses() == 2); model.transform(dataset).registerTempTable("transformed"); - SchemaRDD trans1 = jsql.sql("SELECT rawPrediction, probability FROM transformed"); + DataFrame trans1 = jsql.sql("SELECT rawPrediction, probability FROM transformed"); for (Row row: trans1.collect()) { Vector raw = (Vector)row.get(0); Vector prob = (Vector)row.get(1); @@ -148,7 +135,7 @@ public void logisticRegressionPredictorClassifierMethods() { assert(Math.abs(prob.apply(0) - (1.0 - probFromRaw1)) < eps); } - SchemaRDD trans2 = jsql.sql("SELECT prediction, probability FROM transformed"); + DataFrame trans2 = jsql.sql("SELECT prediction, probability FROM transformed"); for (Row row: trans2.collect()) { double pred = row.getDouble(0); Vector prob = (Vector)row.get(1); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java similarity index 83% rename from mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java rename to mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index d918fc7caf6a0..5bd616e74d86c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.ml.classification; +package org.apache.spark.ml.regression; import java.io.Serializable; -import java.util.ArrayList; import java.util.List; import org.junit.After; @@ -27,32 +26,25 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.regression.LinearRegression; -import org.apache.spark.ml.regression.LinearRegressionModel; import static org.apache.spark.mllib.classification.LogisticRegressionSuite .generateLogisticInputAsList; import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.SchemaRDD; public class JavaLinearRegressionSuite implements Serializable { private transient JavaSparkContext jsc; private transient SQLContext jsql; - private transient SchemaRDD dataset; + private transient DataFrame dataset; private transient JavaRDD datasetRDD; - private double eps = 1e-5; @Before public void setUp() { jsc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); jsql = new SQLContext(jsc); - List points = new ArrayList(); - for (org.apache.spark.mllib.regression.LabeledPoint lp: - generateLogisticInputAsList(1.0, 1.0, 100, 42)) { - points.add(new LabeledPoint(lp.label(), lp.features())); - } + List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); dataset.registerTempTable("dataset"); @@ -70,7 +62,7 @@ public void linearRegressionDefaultParams() { assert(lr.getLabelCol().equals("label")); LinearRegressionModel model = lr.fit(dataset); model.transform(dataset).registerTempTable("prediction"); - SchemaRDD predictions = jsql.sql("SELECT label, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults assert(model.getFeaturesCol().equals("features")); diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index f412622572c1b..b3d1bfcfbee0f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -42,14 +42,18 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { test("logistic regression: default params") { val lr = new LogisticRegression assert(lr.getLabelCol == "label") + assert(lr.getFeaturesCol == "features") + assert(lr.getPredictionCol == "prediction") + assert(lr.getRawPredictionCol == "rawPrediction") + assert(lr.getProbabilityCol == "probability") val model = lr.fit(dataset) model.transform(dataset) - .select('label, 'probability, 'prediction) + .select("label", "probability", "prediction", "rawPrediction") .collect() - // Check defaults assert(model.getThreshold === 0.5) assert(model.getFeaturesCol == "features") assert(model.getPredictionCol == "prediction") + assert(model.getRawPredictionCol == "rawPrediction") assert(model.getProbabilityCol == "probability") } @@ -61,16 +65,6 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { .setThreshold(0.6) .setProbabilityCol("myProbability") val model = lr.fit(dataset) - model.transform(dataset, model.threshold -> 0.8) // overwrite threshold - .select("label", "score", "prediction") - .collect() - } - - test("logistic regression fit and transform with varargs") { - val lr = new LogisticRegression - val model = lr.fit(dataset, lr.maxIter -> 10, lr.regParam -> 1.0) - model.transform(dataset, model.threshold -> 0.8, model.scoreCol -> "probability") - .select("label", "probability", "prediction") assert(model.fittingParamMap.get(lr.maxIter) === Some(10)) assert(model.fittingParamMap.get(lr.regParam) === Some(1.0)) assert(model.fittingParamMap.get(lr.threshold) === Some(0.6)) @@ -79,7 +73,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { // Modify model params, and check that the params worked. model.setThreshold(1.0) val predAllZero = model.transform(dataset) - .select('prediction, 'myProbability) + .select("prediction", "myProbability") .collect() .map { case Row(pred: Double, prob: Vector) => pred } assert(predAllZero.forall(_ === 0), @@ -88,7 +82,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { // Call transform with params, and check that the params worked. val predNotAllZero = model.transform(dataset, model.threshold -> 0.0, model.probabilityCol -> "myProb") - .select('prediction, 'myProb) + .select("prediction", "myProb") .collect() .map { case Row(pred: Double, prob: Vector) => pred } assert(predNotAllZero.exists(_ !== 0.0)) @@ -105,7 +99,6 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { test("logistic regression: Predictor, Classifier methods") { val sqlContext = this.sqlContext - import sqlContext._ val lr = new LogisticRegression val model = lr.fit(dataset) @@ -115,7 +108,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { val results = model.transform(dataset) // Compare rawPrediction with probability - results.select('rawPrediction, 'probability).collect().map { + results.select("rawPrediction", "probability").collect().map { case Row(raw: Vector, prob: Vector) => assert(raw.size === 2) assert(prob.size === 2) @@ -125,7 +118,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { } // Compare prediction with probability - results.select('prediction, 'probability).collect().map { + results.select("prediction", "probability").collect().map { case Row(pred: Double, prob: Vector) => val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 assert(pred == predFromProb) diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index b5876cc96c2b8..bbb44c3e2dfc2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -20,32 +20,27 @@ package org.apache.spark.ml.regression import org.scalatest.FunSuite import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{Row, SQLContext, SchemaRDD} +import org.apache.spark.sql.{DataFrame, SQLContext} class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { @transient var sqlContext: SQLContext = _ - @transient var dataset: SchemaRDD = _ + @transient var dataset: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() sqlContext = new SQLContext(sc) - dataset = sqlContext.createSchemaRDD( + dataset = sqlContext.createDataFrame( sc.parallelize(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42), 2)) } test("linear regression: default params") { - val sqlContext = this.sqlContext - import sqlContext._ val lr = new LinearRegression assert(lr.getLabelCol == "label") val model = lr.fit(dataset) model.transform(dataset) - .select('label, 'prediction) + .select("label", "prediction") .collect() // Check defaults assert(model.getFeaturesCol == "features") @@ -54,8 +49,6 @@ class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { test("linear regression with setters") { // Set params, train, and check as many as we can. - val sqlContext = this.sqlContext - import sqlContext._ val lr = new LinearRegression() .setMaxIter(10) .setRegParam(1.0) From fc6240682ba074bf7fc425d72aba0ab960bd087a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 29 Jan 2015 23:39:27 -0800 Subject: [PATCH 27/30] fixed test suites after last commit --- .../spark/examples/ml/CrossValidatorExample.scala | 2 +- .../spark/examples/ml/DeveloperApiExample.scala | 8 +++++--- .../spark/examples/ml/SimpleParamsExample.scala | 2 +- .../ml/SimpleTextClassificationPipeline.scala | 2 +- .../spark/ml/classification/Classifier.scala | 7 +++---- .../ml/classification/LogisticRegression.scala | 15 +++++++-------- .../classification/ProbabilisticClassifier.scala | 3 +-- .../spark/ml/impl/estimator/Predictor.scala | 3 +-- 8 files changed, 20 insertions(+), 22 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index 0aadd476cba63..a2893f78e0fec 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -101,7 +101,7 @@ object CrossValidatorExample { // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test) - .select('id, 'text, 'probability, 'prediction) + .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => println(s"($id, $text) --> prob=$prob, prediction=$prediction") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 13a860a86afb0..6f68020bf9ee2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -22,7 +22,7 @@ import org.apache.spark.ml.classification.{Classifier, ClassifierParams, Classif import org.apache.spark.ml.param.{Params, IntParam, ParamMap} import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{SchemaRDD, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} /** @@ -68,13 +68,15 @@ object DeveloperApiExample { // Make predictions on test data. val sumPredictions: Double = model.transform(test) - .select('features, 'label, 'prediction) + .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => prediction }.sum assert(sumPredictions == 0.0, "MyLogisticRegression predicted something other than 0, even though all weights are 0!") + + sc.stop() } } @@ -113,7 +115,7 @@ private class MyLogisticRegression // This method is used by fit() override protected def train( - dataset: SchemaRDD, + dataset: DataFrame, paramMap: ParamMap): MyLogisticRegressionModel = { // Extract columns from data using helper method. val oldDataset = extractLabeledPoints(dataset, paramMap) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index ed969f6b64fdc..79ce9fdf7294c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -91,7 +91,7 @@ object SimpleParamsExample { // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. model2.transform(test) - .select('features, 'label, 'myProbability, 'prediction) + .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => println("($features, $label) -> prob=$prob, prediction=$prediction") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index ab93c4847195e..968cb292120d8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -80,7 +80,7 @@ object SimpleTextClassificationPipeline { // Make predictions on test documents. model.transform(test) - .select('id, 'text, 'probability, 'prediction) + .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => println("($id, $text) --> prob=$prob, prediction=$prediction") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index ca791ae612132..40b49e37e076d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -175,15 +175,14 @@ private[ml] object ClassificationModel { val features2raw: FeaturesType => Vector = model.predictRaw tmpData = tmpData.select($"*", callUDF(features2raw, new VectorUDT, - tmpData(map(model.featuresCol))).as(map(model.rawPredictionCol))) + col(map(model.featuresCol))).as(map(model.rawPredictionCol))) numColsOutput += 1 if (map(model.predictionCol) != "") { val raw2pred: Vector => Double = (rawPred) => { rawPred.toArray.zipWithIndex.maxBy(_._1)._2 } tmpData = tmpData.select($"*", - callUDF(raw2pred, DoubleType, - tmpData(map(model.rawPredictionCol))).as(map(model.predictionCol))) + callUDF(raw2pred, col(map(model.rawPredictionCol))).as(map(model.predictionCol))) numColsOutput += 1 } } else if (map(model.predictionCol) != "") { @@ -191,7 +190,7 @@ private[ml] object ClassificationModel { val features2pred: FeaturesType => Double = model.predict tmpData = tmpData.select($"*", callUDF(features2pred, DoubleType, - tmpData(map(model.featuresCol))).as(map(model.predictionCol))) + col(map(model.featuresCol))).as(map(model.predictionCol))) numColsOutput += 1 } (numColsOutput, tmpData) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 293062fc45915..4492c40aa2bfc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -20,10 +20,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Dsl._ -import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -120,7 +119,7 @@ class LogisticRegressionModel private[ml] ( if (map(rawPredictionCol) != "") { val features2raw: Vector => Vector = predictRaw tmpData = tmpData.select($"*", - callUDF(features2raw, new VectorUDT, tmpData(map(featuresCol))).as(map(rawPredictionCol))) + callUDF(features2raw, col(map(featuresCol))).as(map(rawPredictionCol))) numColsOutput += 1 } if (map(probabilityCol) != "") { @@ -130,11 +129,11 @@ class LogisticRegressionModel private[ml] ( Vectors.dense(1.0 - prob1, prob1) } tmpData = tmpData.select($"*", - callUDF(raw2prob, new VectorUDT, tmpData(map(rawPredictionCol))).as(map(probabilityCol))) + callUDF(raw2prob, col(map(rawPredictionCol))).as(map(probabilityCol))) } else { val features2prob: Vector => Vector = predictProbabilities tmpData = tmpData.select($"*", - callUDF(features2prob, new VectorUDT, tmpData(map(featuresCol))).as(map(probabilityCol))) + callUDF(features2prob, col(map(featuresCol))).as(map(probabilityCol))) } numColsOutput += 1 } @@ -145,18 +144,18 @@ class LogisticRegressionModel private[ml] ( if (probs(1) > t) 1.0 else 0.0 } tmpData = tmpData.select($"*", - callUDF(predict, DoubleType, tmpData(map(probabilityCol))).as(map(predictionCol))) + callUDF(predict, col(map(probabilityCol))).as(map(predictionCol))) } else if (map(rawPredictionCol) != "") { val predict: Vector => Double = (rawPreds) => { val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) if (prob1 > t) 1.0 else 0.0 } tmpData = tmpData.select($"*", - callUDF(predict, DoubleType, tmpData(map(rawPredictionCol))).as(map(predictionCol))) + callUDF(predict, col(map(rawPredictionCol))).as(map(predictionCol))) } else { val predict: Vector => Double = this.predict tmpData = tmpData.select($"*", - callUDF(predict, DoubleType, tmpData(map(featuresCol))).as(map(predictionCol))) + callUDF(predict, col(map(featuresCol))).as(map(predictionCol))) } numColsOutput += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 330571cf7b91f..f7b8afdc9d380 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -117,8 +117,7 @@ abstract class ProbabilisticClassificationModel[ tmpModel.predictProbabilities(features) } outputData.select($"*", - callUDF(features2probs, new VectorUDT, - outputData(map(featuresCol))).as(map(probabilityCol))) + callUDF(features2probs, new VectorUDT, col(map(featuresCol))).as(map(probabilityCol))) } else { if (numColsOutput == 0) { this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index cdf458f584ae4..59a4e44b13fda 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -203,8 +203,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, val pred: FeaturesType => Double = (features) => { tmpModel.predict(features) } - dataset.select($"*", - callUDF(pred, DoubleType, dataset(map(featuresCol))).as(map(predictionCol))) + dataset.select($"*", callUDF(pred, DoubleType, col(map(featuresCol))).as(map(predictionCol))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + " since no output columns were set.") From 8316d5ed8a9e810edbc5a202e5a7e8337cee9934 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 5 Feb 2015 13:29:56 -0800 Subject: [PATCH 28/30] fixes after rebasing on master --- .../examples/ml/DeveloperApiExample.scala | 6 +- .../examples/ml/SimpleParamsExample.scala | 2 +- .../spark/ml/classification/Classifier.scala | 4 +- .../classification/LogisticRegression.scala | 64 ------------------- 4 files changed, 6 insertions(+), 70 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 6f68020bf9ee2..7a19bcd3c8d77 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -40,10 +40,10 @@ object DeveloperApiExample { val conf = new SparkConf().setAppName("DeveloperApiExample") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) - import sqlContext._ + import sqlContext.implicits._ // Prepare training data. - val training = sparkContext.parallelize(Seq( + val training = sc.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -61,7 +61,7 @@ object DeveloperApiExample { val model = lr.fit(training) // Prepare test data. - val test = sparkContext.parallelize(Seq( + val test = sc.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 79ce9fdf7294c..80c9f5ff5781e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -81,7 +81,7 @@ object SimpleParamsExample { println("Model 2 was fit using parameters: " + model2.fittingParamMap) // Prepare test data. - val test = sparkContext.parallelize(Seq( + val test = sc.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 40b49e37e076d..a4fbf04e03112 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -181,8 +181,8 @@ private[ml] object ClassificationModel { val raw2pred: Vector => Double = (rawPred) => { rawPred.toArray.zipWithIndex.maxBy(_._1)._2 } - tmpData = tmpData.select($"*", - callUDF(raw2pred, col(map(model.rawPredictionCol))).as(map(model.predictionCol))) + tmpData = tmpData.select($"*", callUDF(raw2pred, DoubleType, + col(map(model.rawPredictionCol))).as(map(model.predictionCol))) numColsOutput += 1 } } else if (map(model.predictionCol) != "") { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 4492c40aa2bfc..3246c9beae241 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -22,7 +22,6 @@ import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Dsl._ import org.apache.spark.storage.StorageLevel @@ -103,69 +102,6 @@ class LogisticRegressionModel private[ml] ( 1.0 / (1.0 + math.exp(-m)) } - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - - val map = this.paramMap ++ paramMap - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - // rawPrediction (-margin, margin) - // probability (1.0-score, score) - // prediction (max margin) - var tmpData = dataset - var numColsOutput = 0 - if (map(rawPredictionCol) != "") { - val features2raw: Vector => Vector = predictRaw - tmpData = tmpData.select($"*", - callUDF(features2raw, col(map(featuresCol))).as(map(rawPredictionCol))) - numColsOutput += 1 - } - if (map(probabilityCol) != "") { - if (map(rawPredictionCol) != "") { - val raw2prob: Vector => Vector = (rawPreds) => { - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - Vectors.dense(1.0 - prob1, prob1) - } - tmpData = tmpData.select($"*", - callUDF(raw2prob, col(map(rawPredictionCol))).as(map(probabilityCol))) - } else { - val features2prob: Vector => Vector = predictProbabilities - tmpData = tmpData.select($"*", - callUDF(features2prob, col(map(featuresCol))).as(map(probabilityCol))) - } - numColsOutput += 1 - } - if (map(predictionCol) != "") { - val t = map(threshold) - if (map(probabilityCol) != "") { - val predict: Vector => Double = (probs) => { - if (probs(1) > t) 1.0 else 0.0 - } - tmpData = tmpData.select($"*", - callUDF(predict, col(map(probabilityCol))).as(map(predictionCol))) - } else if (map(rawPredictionCol) != "") { - val predict: Vector => Double = (rawPreds) => { - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - if (prob1 > t) 1.0 else 0.0 - } - tmpData = tmpData.select($"*", - callUDF(predict, col(map(rawPredictionCol))).as(map(predictionCol))) - } else { - val predict: Vector => Double = this.predict - tmpData = tmpData.select($"*", - callUDF(predict, col(map(featuresCol))).as(map(predictionCol))) - } - numColsOutput += 1 - } - if (numColsOutput == 0) { - this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + - " since no output columns were set.") - } - tmpData - } - override val numClasses: Int = 2 /** From fec348ac0a50688ed916699bc892e269730c1445 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 5 Feb 2015 17:02:54 -0800 Subject: [PATCH 29/30] Added JavaDeveloperApiExample.java and fixed other issues: Made developer API private[spark] for now. Added constructors Java can understand to specialized Param types. --- .../ml/JavaCrossValidatorExample.java | 2 + .../examples/ml/JavaDeveloperApiExample.java | 217 ++++++++++++++++++ .../examples/ml/JavaSimpleParamsExample.java | 2 + .../JavaSimpleTextClassificationPipeline.java | 2 + .../spark/ml/classification/Classifier.scala | 13 +- .../classification/LogisticRegression.scala | 73 +++++- .../ProbabilisticClassifier.scala | 8 +- .../spark/ml/impl/estimator/Predictor.scala | 10 +- .../org/apache/spark/ml/param/params.scala | 20 +- .../spark/ml/regression/Regressor.scala | 12 +- 10 files changed, 343 insertions(+), 16 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index 7fc44739b6ca7..5041e0b6d34b0 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -121,5 +121,7 @@ public static void main(String[] args) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java new file mode 100644 index 0000000000000..42d4d7d0bef26 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -0,0 +1,217 @@ +/* + * 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.examples.ml; + +import java.util.List; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.Classifier; +import org.apache.spark.ml.classification.ClassificationModel; +import org.apache.spark.ml.param.IntParam; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.param.Params; +import org.apache.spark.ml.param.Params$; +import org.apache.spark.mllib.linalg.BLAS; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; + + +/** + * A simple example demonstrating how to write your own learning algorithm using Estimator, + * Transformer, and other abstractions. + * This mimics {@link org.apache.spark.ml.classification.LogisticRegression}. + * + * Run with + *
+ * bin/run-example ml.JavaDeveloperApiExample
+ * 
+ */ +public class JavaDeveloperApiExample { + + public static void main(String[] args) throws Exception { + SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + SQLContext jsql = new SQLContext(jsc); + + // Prepare training data. + List localTraining = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); + DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + + // Create a LogisticRegression instance. This instance is an Estimator. + MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); + // Print out the parameters, documentation, and any default values. + System.out.println("MyJavaLogisticRegression parameters:\n" + lr.explainParams() + "\n"); + + // We may set parameters using setter methods. + lr.setMaxIter(10); + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + MyJavaLogisticRegressionModel model = lr.fit(training); + + // Prepare test data. + List localTest = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); + DataFrame test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + DataFrame results = model.transform(test); + double sumPredictions = 0; + for (Row r : results.select("features", "label", "prediction").collect()) { + sumPredictions += r.getDouble(2); + } + if (sumPredictions != 0.0) { + throw new Exception("MyJavaLogisticRegression predicted something other than 0," + + " even though all weights are 0!"); + } + + jsc.stop(); + } +} + +/** + * Example of defining a type of {@link Classifier}. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +class MyJavaLogisticRegression + extends Classifier + implements Params { + + /** + * Param for max number of iterations + *

+ * NOTE: The usual way to add a parameter to a model or algorithm is to include: + * - val myParamName: ParamType + * - def getMyParamName + * - def setMyParamName + */ + IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); + + int getMaxIter() { return (int)get(maxIter); } + + public MyJavaLogisticRegression() { + setMaxIter(100); + } + + // The parameter setter is in this class since it should return type MyJavaLogisticRegression. + MyJavaLogisticRegression setMaxIter(int value) { + return (MyJavaLogisticRegression)set(maxIter, value); + } + + // This method is used by fit(). + // In Java, we have to make it public since Java does not understand Scala's protected modifier. + public MyJavaLogisticRegressionModel train(DataFrame dataset, ParamMap paramMap) { + // Extract columns from data using helper method. + JavaRDD oldDataset = extractLabeledPoints(dataset, paramMap).toJavaRDD(); + + // Do learning to estimate the weight vector. + int numFeatures = oldDataset.take(1).get(0).features().size(); + Vector weights = Vectors.zeros(numFeatures); // Learning would happen here. + + // Create a model, and return it. + return new MyJavaLogisticRegressionModel(this, paramMap, weights); + } +} + +/** + * Example of defining a type of {@link ClassificationModel}. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +class MyJavaLogisticRegressionModel + extends ClassificationModel implements Params { + + private MyJavaLogisticRegression parent_; + public MyJavaLogisticRegression parent() { return parent_; } + + private ParamMap fittingParamMap_; + public ParamMap fittingParamMap() { return fittingParamMap_; } + + private Vector weights_; + public Vector weights() { return weights_; } + + public MyJavaLogisticRegressionModel( + MyJavaLogisticRegression parent_, + ParamMap fittingParamMap_, + Vector weights_) { + this.parent_ = parent_; + this.fittingParamMap_ = fittingParamMap_; + this.weights_ = weights_; + } + + // This uses the default implementation of transform(), which reads column "features" and outputs + // columns "prediction" and "rawPrediction." + + // This uses the default implementation of predict(), which chooses the label corresponding to + // the maximum value returned by [[predictRaw()]]. + + /** + * Raw prediction for each possible label. + * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives + * a measure of confidence in each possible label (where larger = more confident). + * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * + * @return vector where element i is the raw prediction for label i. + * This raw prediction may be any real number, where a larger value indicates greater + * confidence for that label. + * + * In Java, we have to make this method public since Java does not understand Scala's protected + * modifier. + */ + public Vector predictRaw(Vector features) { + double margin = BLAS.dot(features, weights_); + // There are 2 classes (binary classification), so we return a length-2 vector, + // where index i corresponds to class i (i = 0, 1). + return Vectors.dense(-margin, margin); + } + + /** + * Number of classes the label can take. 2 indicates binary classification. + */ + public int numClasses() { return 2; } + + /** + * Create a copy of the model. + * The copy is shallow, except for the embedded paramMap, which gets a deep copy. + *

+ * This is used for the defaul implementation of [[transform()]]. + * + * In Java, we have to make this method public since Java does not understand Scala's protected + * modifier. + */ + public MyJavaLogisticRegressionModel copy() { + MyJavaLogisticRegressionModel m = + new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_); + Params$.MODULE$.inheritValues(this.paramMap(), this, m); + return m; + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index 98677d0a4a67b..cc69e6315fdda 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -107,5 +107,7 @@ public static void main(String[] args) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index f27550e7337dd..d929f1ad2014a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -88,5 +88,7 @@ public static void main(String[] args) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + + jsc.stop(); } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index a4fbf04e03112..579b96a83c938 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -29,9 +29,11 @@ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** * :: DeveloperApi :: * Params for classification. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @DeveloperApi -trait ClassifierParams extends PredictorParams +private[spark] trait ClassifierParams extends PredictorParams with HasRawPredictionCol { override protected def validateAndTransformSchema( @@ -53,9 +55,11 @@ trait ClassifierParams extends PredictorParams * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class Classifier[ +private[spark] abstract class Classifier[ FeaturesType, Learner <: Classifier[FeaturesType, Learner, M], M <: ClassificationModel[FeaturesType, M]] @@ -75,8 +79,11 @@ abstract class Classifier[ * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent +private[spark] abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { @@ -161,7 +168,7 @@ private[ml] object ClassificationModel { * should already be done. * @return (number of columns added, transformed dataset) */ - private[ml] def transformColumnsImpl[FeaturesType]( + def transformColumnsImpl[FeaturesType]( dataset: DataFrame, model: ClassificationModel[FeaturesType, _], map: ParamMap): (Int, DataFrame) = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 3246c9beae241..c146fe244c66e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -20,8 +20,10 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -102,6 +104,74 @@ class LogisticRegressionModel private[ml] ( 1.0 / (1.0 + math.exp(-m)) } + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + // This is overridden (a) to be more efficient (avoiding re-computing values when creating + // multiple output columns) and (b) to handle threshold, which the abstractions do not use. + // TODO: We should abstract away the steps defined by UDFs below so that the abstractions + // can call whichever UDFs are needed to create the output columns. + + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + + val map = this.paramMap ++ paramMap + + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + // rawPrediction (-margin, margin) + // probability (1.0-score, score) + // prediction (max margin) + var tmpData = dataset + var numColsOutput = 0 + if (map(rawPredictionCol) != "") { + val features2raw: Vector => Vector = (features) => predictRaw(features) + tmpData = tmpData.select($"*", + callUDF(features2raw, new VectorUDT, col(map(featuresCol))).as(map(rawPredictionCol))) + numColsOutput += 1 + } + if (map(probabilityCol) != "") { + if (map(rawPredictionCol) != "") { + val raw2prob: Vector => Vector = { (rawPreds: Vector) => + val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) + Vectors.dense(1.0 - prob1, prob1) + } + tmpData = tmpData.select($"*", + callUDF(raw2prob, new VectorUDT, col(map(rawPredictionCol))).as(map(probabilityCol))) + } else { + val features2prob: Vector => Vector = (features: Vector) => predictProbabilities(features) + tmpData = tmpData.select($"*", + callUDF(features2prob, new VectorUDT, col(map(featuresCol))).as(map(probabilityCol))) + } + numColsOutput += 1 + } + if (map(predictionCol) != "") { + val t = map(threshold) + if (map(probabilityCol) != "") { + val predict: Vector => Double = { probs: Vector => + if (probs(1) > t) 1.0 else 0.0 + } + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, col(map(probabilityCol))).as(map(predictionCol))) + } else if (map(rawPredictionCol) != "") { + val predict: Vector => Double = { rawPreds: Vector => + val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) + if (prob1 > t) 1.0 else 0.0 + } + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, col(map(rawPredictionCol))).as(map(predictionCol))) + } else { + val predict: Vector => Double = (features: Vector) => this.predict(features) + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, col(map(featuresCol))).as(map(predictionCol))) + } + numColsOutput += 1 + } + if (numColsOutput == 0) { + this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + + " since no output columns were set.") + } + tmpData + } + override val numClasses: Int = 2 /** @@ -109,6 +179,7 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override protected def predict(features: Vector): Double = { + println(s"LR.predict with threshold: ${paramMap(threshold)}") if (score(features) > paramMap(threshold)) 1 else 0 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index f7b8afdc9d380..fd41d077f7cad 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -51,9 +51,11 @@ private[classification] trait ProbabilisticClassifierParams * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class ProbabilisticClassifier[ +private[spark] abstract class ProbabilisticClassifier[ FeaturesType, Learner <: ProbabilisticClassifier[FeaturesType, Learner, M], M <: ProbabilisticClassificationModel[FeaturesType, M]] @@ -71,9 +73,11 @@ abstract class ProbabilisticClassifier[ * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class ProbabilisticClassificationModel[ +private[spark] abstract class ProbabilisticClassificationModel[ FeaturesType, M <: ProbabilisticClassificationModel[FeaturesType, M]] extends ClassificationModel[FeaturesType, M] with ProbabilisticClassifierParams { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 59a4e44b13fda..d3875b733b4c9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -32,9 +32,11 @@ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} * :: DeveloperApi :: * * Trait for parameters for prediction (regression and classification). + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @DeveloperApi -trait PredictorParams extends Params +private[spark] trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol { /** @@ -73,6 +75,8 @@ trait PredictorParams extends Params * parameter to specify the concrete type. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent abstract class Predictor[ @@ -149,9 +153,11 @@ abstract class Predictor[ * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] +private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { def setFeaturesCol(value: String): M = set(featuresCol, value).asInstanceOf[M] diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index c4f98a7222d06..ae3481ef2346d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -67,37 +67,47 @@ class Param[T] ( // specialize primitive-typed params because Java doesn't recognize scala.Double, scala.Int, ... /** Specialized version of [[Param[Double]]] for Java. */ -class DoubleParam(parent: Params, name: String, doc: String, defaultValue: Option[Double] = None) +class DoubleParam(parent: Params, name: String, doc: String, defaultValue: Option[Double]) extends Param[Double](parent, name, doc, defaultValue) { + def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + override def w(value: Double): ParamPair[Double] = super.w(value) } /** Specialized version of [[Param[Int]]] for Java. */ -class IntParam(parent: Params, name: String, doc: String, defaultValue: Option[Int] = None) +class IntParam(parent: Params, name: String, doc: String, defaultValue: Option[Int]) extends Param[Int](parent, name, doc, defaultValue) { + def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + override def w(value: Int): ParamPair[Int] = super.w(value) } /** Specialized version of [[Param[Float]]] for Java. */ -class FloatParam(parent: Params, name: String, doc: String, defaultValue: Option[Float] = None) +class FloatParam(parent: Params, name: String, doc: String, defaultValue: Option[Float]) extends Param[Float](parent, name, doc, defaultValue) { + def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + override def w(value: Float): ParamPair[Float] = super.w(value) } /** Specialized version of [[Param[Long]]] for Java. */ -class LongParam(parent: Params, name: String, doc: String, defaultValue: Option[Long] = None) +class LongParam(parent: Params, name: String, doc: String, defaultValue: Option[Long]) extends Param[Long](parent, name, doc, defaultValue) { + def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + override def w(value: Long): ParamPair[Long] = super.w(value) } /** Specialized version of [[Param[Boolean]]] for Java. */ -class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Option[Boolean] = None) +class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Option[Boolean]) extends Param[Boolean](parent, name, doc, defaultValue) { + def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index dca849f44270f..d679085eeafe1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -24,9 +24,11 @@ import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, Predictor * :: DeveloperApi :: * Params for regression. * Currently empty, but may add functionality later. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @DeveloperApi -trait RegressorParams extends PredictorParams +private[spark] trait RegressorParams extends PredictorParams /** * :: AlphaComponent :: @@ -36,9 +38,11 @@ trait RegressorParams extends PredictorParams * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class Regressor[ +private[spark] abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], M <: RegressionModel[FeaturesType, M]] @@ -55,9 +59,11 @@ abstract class Regressor[ * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam M Concrete Model type. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] +private[spark] abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with RegressorParams { /** From 405bfb8e4e54f1dd2619c1d9d35698aa9ab8efc3 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 5 Feb 2015 18:16:31 -0800 Subject: [PATCH 30/30] Last edits based on code review. Small cleanups --- .../apache/spark/examples/ml/DeveloperApiExample.scala | 3 +++ .../apache/spark/ml/classification/Classifier.scala | 10 +++++----- .../ml/classification/ProbabilisticClassifier.scala | 8 ++++---- .../org/apache/spark/ml/impl/estimator/Predictor.scala | 2 +- .../main/scala/org/apache/spark/ml/param/params.scala | 4 +++- .../scala/org/apache/spark/mllib/linalg/Vectors.scala | 4 +++- 6 files changed, 19 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 7a19bcd3c8d77..aed44238939c7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -94,6 +94,9 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * - val myParamName: ParamType * - def getMyParamName * - def setMyParamName + * Here, we have a trait to be mixed in with the Estimator and Model (MyLogisticRegression + * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator + * class since the maxIter parameter is only used during training (not in the Model). */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") def getMaxIter: Int = get(maxIter) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 579b96a83c938..1bf8eb4640d11 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -53,7 +53,7 @@ private[spark] trait ClassifierParams extends PredictorParams * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] - * @tparam Learner Concrete Estimator type + * @tparam E Concrete Estimator type * @tparam M Concrete Model type * * NOTE: This is currently private[spark] but will be made public later once it is stabilized. @@ -61,13 +61,13 @@ private[spark] trait ClassifierParams extends PredictorParams @AlphaComponent private[spark] abstract class Classifier[ FeaturesType, - Learner <: Classifier[FeaturesType, Learner, M], + E <: Classifier[FeaturesType, E, M], M <: ClassificationModel[FeaturesType, M]] - extends Predictor[FeaturesType, Learner, M] + extends Predictor[FeaturesType, E, M] with ClassifierParams { - def setRawPredictionCol(value: String): Learner = - set(rawPredictionCol, value).asInstanceOf[Learner] + def setRawPredictionCol(value: String): E = + set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index fd41d077f7cad..1202528ca654e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -49,7 +49,7 @@ private[classification] trait ProbabilisticClassifierParams * Single-label binary or multiclass classifier which can output class conditional probabilities. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] - * @tparam Learner Concrete Estimator type + * @tparam E Concrete Estimator type * @tparam M Concrete Model type * * NOTE: This is currently private[spark] but will be made public later once it is stabilized. @@ -57,11 +57,11 @@ private[classification] trait ProbabilisticClassifierParams @AlphaComponent private[spark] abstract class ProbabilisticClassifier[ FeaturesType, - Learner <: ProbabilisticClassifier[FeaturesType, Learner, M], + E <: ProbabilisticClassifier[FeaturesType, E, M], M <: ProbabilisticClassificationModel[FeaturesType, M]] - extends Classifier[FeaturesType, Learner, M] with ProbabilisticClassifierParams { + extends Classifier[FeaturesType, E, M] with ProbabilisticClassifierParams { - def setProbabilityCol(value: String): Learner = set(probabilityCol, value).asInstanceOf[Learner] + def setProbabilityCol(value: String): E = set(probabilityCol, value).asInstanceOf[E] } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index d3875b733b4c9..89b53f3890ea3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -79,7 +79,7 @@ private[spark] trait PredictorParams extends Params * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @AlphaComponent -abstract class Predictor[ +private[spark] abstract class Predictor[ FeaturesType, Learner <: Predictor[FeaturesType, Learner, M], M <: PredictionModel[FeaturesType, M]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index ae3481ef2346d..17ece897a6c55 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -224,9 +224,11 @@ trait Params extends Identifiable with Serializable { * :: DeveloperApi :: * * Helper functionality for developers. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @DeveloperApi -object Params { +private[spark] object Params { /** * Copies parameter values from the parent estimator to the child model it produced. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 6a5259d848249..480bbfb5fe94a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -115,9 +115,11 @@ sealed trait Vector extends Serializable { * * User-defined type for [[Vector]] which allows easy interaction with SQL * via [[org.apache.spark.sql.DataFrame]]. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ @DeveloperApi -class VectorUDT extends UserDefinedType[Vector] { +private[spark] class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { // type: 0 = sparse, 1 = dense