Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ import org.apache.spark.sql.functions.{col, lit}
/** Params for linear SVM Classifier. */
private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam
with HasMaxIter with HasFitIntercept with HasTol with HasStandardization with HasWeightCol
with HasAggregationDepth {
with HasAggregationDepth with HasThreshold {

/**
* Param for threshold in binary classification prediction.
Expand All @@ -53,11 +53,8 @@ private[classification] trait LinearSVCParams extends ClassifierParams with HasR
*
* @group param
*/
final val threshold: DoubleParam = new DoubleParam(this, "threshold",
final override val threshold: DoubleParam = new DoubleParam(this, "threshold",
"threshold in binary classification prediction applied to rawPrediction")

/** @group getParam */
def getThreshold: Double = $(threshold)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -366,6 +366,7 @@ class LogisticRegression @Since("1.2.0") (

@Since("1.5.0")
override def setThreshold(value: Double): this.type = super.setThreshold(value)
setDefault(threshold -> 0.5)

@Since("1.5.0")
override def getThreshold: Double = super.getThreshold
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.ml.feature

import org.apache.hadoop.fs.Path

import org.apache.spark.SparkContext
import org.apache.spark.annotation.Since
import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,8 @@ private[shared] object SharedParamsCodeGen {
Some("\"probability\"")),
ParamDesc[String]("varianceCol", "Column name for the biased sample variance of prediction"),
ParamDesc[Double]("threshold",
"threshold in binary classification prediction, in range [0, 1]", Some("0.5"),
isValid = "ParamValidators.inRange(0, 1)", finalMethods = false),
"threshold in binary classification prediction, in range [0, 1]",
isValid = "ParamValidators.inRange(0, 1)", finalMethods = false, finalFields = false),
ParamDesc[Array[Double]]("thresholds", "Thresholds in multi-class classification" +
" to adjust the probability of predicting each class." +
" Array must have length equal to the number of classes, with values > 0" +
Expand Down Expand Up @@ -77,7 +77,7 @@ private[shared] object SharedParamsCodeGen {
ParamDesc[Double]("tol", "the convergence tolerance for iterative algorithms (>= 0)",
isValid = "ParamValidators.gtEq(0)"),
ParamDesc[Double]("stepSize", "Step size to be used for each iteration of optimization (>" +
" 0)", isValid = "ParamValidators.gt(0)"),
" 0)", isValid = "ParamValidators.gt(0)", finalFields = false),
ParamDesc[String]("weightCol", "weight column name. If this is not set or empty, we treat " +
"all instance weights as 1.0"),
ParamDesc[String]("solver", "the solver algorithm for optimization", finalFields = false),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,9 +162,7 @@ private[ml] trait HasThreshold extends Params {
* Param for threshold in binary classification prediction, in range [0, 1].
* @group param
*/
final val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction, in range [0, 1]", ParamValidators.inRange(0, 1))

setDefault(threshold, 0.5)
val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction, in range [0, 1]", ParamValidators.inRange(0, 1))

/** @group getParam */
def getThreshold: Double = $(threshold)
Expand Down Expand Up @@ -352,7 +350,7 @@ private[ml] trait HasStepSize extends Params {
* Param for Step size to be used for each iteration of optimization (> 0).
* @group param
*/
final val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size to be used for each iteration of optimization (> 0)", ParamValidators.gt(0))
val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size to be used for each iteration of optimization (> 0)", ParamValidators.gt(0))

/** @group getParam */
final def getStepSize: Double = $(stepSize)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -458,7 +458,7 @@ private[ml] trait RandomForestRegressorParams
*
* Note: Marked as private and DeveloperApi since this may be made public in the future.
*/
private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter {
private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasStepSize {

/* TODO: Add this doc when we add this param. SPARK-7132
* Threshold for stopping early when runWithValidation is used.
Expand All @@ -484,13 +484,10 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter {
* (default = 0.1)
* @group param
*/
final val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size " +
final override val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size " +
"(a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each estimator.",
ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true))

/** @group getParam */
final def getStepSize: Double = $(stepSize)

/**
* @deprecated This method is deprecated and will be removed in 3.0.0.
* @group setParam
Expand Down
19 changes: 6 additions & 13 deletions python/pyspark/ml/classification.py
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ def numClasses(self):
@inherit_doc
class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,
HasRegParam, HasTol, HasRawPredictionCol, HasFitIntercept, HasStandardization,
HasWeightCol, HasAggregationDepth, JavaMLWritable, JavaMLReadable):
HasWeightCol, HasAggregationDepth, HasThreshold, JavaMLWritable, JavaMLReadable):
"""
.. note:: Experimental

Expand Down Expand Up @@ -153,18 +153,6 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre
def _create_model(self, java_model):
return LinearSVCModel(java_model)

def setThreshold(self, value):
"""
Sets the value of :py:attr:`threshold`.
"""
return self._set(threshold=value)

def getThreshold(self):
"""
Gets the value of threshold or its default value.
"""
return self.getOrDefault(self.threshold)


class LinearSVCModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable):
"""
Expand Down Expand Up @@ -1030,6 +1018,11 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol
"Supported options: " + ", ".join(GBTParams.supportedLossTypes),
typeConverter=TypeConverters.toString)

stepSize = Param(Params._dummy(), "stepSize",
"Step size (a.k.a. learning rate) in interval (0, 1] for shrinking " +
"the contribution of each estimator.",
typeConverter=TypeConverters.toFloat)

@keyword_only
def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction",
maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0,
Expand Down
5 changes: 5 additions & 0 deletions python/pyspark/ml/regression.py
Original file line number Diff line number Diff line change
Expand Up @@ -1014,6 +1014,11 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,
"Supported options: " + ", ".join(GBTParams.supportedLossTypes),
typeConverter=TypeConverters.toString)

stepSize = Param(Params._dummy(), "stepSize",
"Step size (a.k.a. learning rate) in interval (0, 1] for shrinking " +
"the contribution of each estimator.",
typeConverter=TypeConverters.toFloat)

@keyword_only
def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction",
maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0,
Expand Down