aboutsummaryrefslogtreecommitdiff
path: root/python
diff options
context:
space:
mode:
authorHolden Karau <holden@us.ibm.com>2016-01-06 10:43:03 -0800
committerJoseph K. Bradley <joseph@databricks.com>2016-01-06 10:43:03 -0800
commit3b29004d2439c03a7d9bfdf7c2edd757d3d8c240 (patch)
tree66fb557cad74f3e1507ecbcd1113bc913fac8fbc /python
parent9061e777fdcd5767718808e325e8953d484aa761 (diff)
downloadspark-3b29004d2439c03a7d9bfdf7c2edd757d3d8c240.tar.gz
spark-3b29004d2439c03a7d9bfdf7c2edd757d3d8c240.tar.bz2
spark-3b29004d2439c03a7d9bfdf7c2edd757d3d8c240.zip
[SPARK-7675][ML][PYSPARK] sparkml params type conversion
From JIRA: Currently, PySpark wrappers for spark.ml Scala classes are brittle when accepting Param types. E.g., Normalizer's "p" param cannot be set to "2" (an integer); it must be set to "2.0" (a float). Fixing this is not trivial since there does not appear to be a natural place to insert the conversion before Python wrappers call Java's Params setter method. A possible fix will be to include a method "_checkType" to PySpark's Param class which checks the type, prints an error if needed, and converts types when relevant (e.g., int to float, or scipy matrix to array). The Java wrapper method which copies params to Scala can call this method when available. This fix instead checks the types at set time since I think failing sooner is better, but I can switch it around to check at copy time if that would be better. So far this only converts int to float and other conversions (like scipymatrix to array) are left for the future. Author: Holden Karau <holden@us.ibm.com> Closes #9581 from holdenk/SPARK-7675-PySpark-sparkml-Params-type-conversion.
Diffstat (limited to 'python')
-rw-r--r--python/pyspark/ml/param/__init__.py22
-rw-r--r--python/pyspark/ml/param/_shared_params_code_gen.py63
-rw-r--r--python/pyspark/ml/param/shared.py144
-rw-r--r--python/pyspark/ml/tests.py22
4 files changed, 148 insertions, 103 deletions
diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py
index 35c9b776a3..92ce96aa3c 100644
--- a/python/pyspark/ml/param/__init__.py
+++ b/python/pyspark/ml/param/__init__.py
@@ -32,12 +32,13 @@ class Param(object):
.. versionadded:: 1.3.0
"""
- def __init__(self, parent, name, doc):
+ def __init__(self, parent, name, doc, expectedType=None):
if not isinstance(parent, Identifiable):
raise TypeError("Parent must be an Identifiable but got type %s." % type(parent))
self.parent = parent.uid
self.name = str(name)
self.doc = str(doc)
+ self.expectedType = expectedType
def __str__(self):
return str(self.parent) + "__" + self.name
@@ -247,7 +248,24 @@ class Params(Identifiable):
Sets user-supplied params.
"""
for param, value in kwargs.items():
- self._paramMap[getattr(self, param)] = value
+ p = getattr(self, param)
+ if p.expectedType is None or type(value) == p.expectedType or value is None:
+ self._paramMap[getattr(self, param)] = value
+ else:
+ try:
+ # Try and do "safe" conversions that don't lose information
+ if p.expectedType == float:
+ self._paramMap[getattr(self, param)] = float(value)
+ # Python 3 unified long & int
+ elif p.expectedType == int and type(value).__name__ == 'long':
+ self._paramMap[getattr(self, param)] = value
+ else:
+ raise Exception(
+ "Provided type {0} incompatible with type {1} for param {2}"
+ .format(type(value), p.expectedType, p))
+ except ValueError:
+ raise Exception(("Failed to convert {0} to type {1} for param {2}"
+ .format(type(value), p.expectedType, p)))
return self
def _setDefault(self, **kwargs):
diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py
index 0528dc1e3a..82855bc4c7 100644
--- a/python/pyspark/ml/param/_shared_params_code_gen.py
+++ b/python/pyspark/ml/param/_shared_params_code_gen.py
@@ -38,7 +38,7 @@ header = """#
# python _shared_params_code_gen.py > shared.py
-def _gen_param_header(name, doc, defaultValueStr):
+def _gen_param_header(name, doc, defaultValueStr, expectedType):
"""
Generates the header part for shared variables
@@ -51,22 +51,26 @@ def _gen_param_header(name, doc, defaultValueStr):
"""
# a placeholder to make it appear in the generated doc
- $name = Param(Params._dummy(), "$name", "$doc")
+ $name = Param(Params._dummy(), "$name", "$doc", $expectedType)
def __init__(self):
super(Has$Name, self).__init__()
#: param for $doc
- self.$name = Param(self, "$name", "$doc")'''
+ self.$name = Param(self, "$name", "$doc", $expectedType)'''
if defaultValueStr is not None:
template += '''
self._setDefault($name=$defaultValueStr)'''
Name = name[0].upper() + name[1:]
+ expectedTypeName = str(expectedType)
+ if expectedType is not None:
+ expectedTypeName = expectedType.__name__
return template \
.replace("$name", name) \
.replace("$Name", Name) \
.replace("$doc", doc) \
- .replace("$defaultValueStr", str(defaultValueStr))
+ .replace("$defaultValueStr", str(defaultValueStr)) \
+ .replace("$expectedType", expectedTypeName)
def _gen_param_code(name, doc, defaultValueStr):
@@ -84,7 +88,7 @@ def _gen_param_code(name, doc, defaultValueStr):
"""
Sets the value of :py:attr:`$name`.
"""
- self._paramMap[self.$name] = value
+ self._set($name=value)
return self
def get$Name(self):
@@ -105,44 +109,45 @@ if __name__ == "__main__":
print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n")
print("from pyspark.ml.param import Param, Params\n\n")
shared = [
- ("maxIter", "max number of iterations (>= 0).", None),
- ("regParam", "regularization parameter (>= 0).", None),
- ("featuresCol", "features column name.", "'features'"),
- ("labelCol", "label column name.", "'label'"),
- ("predictionCol", "prediction column name.", "'prediction'"),
+ ("maxIter", "max number of iterations (>= 0).", None, int),
+ ("regParam", "regularization parameter (>= 0).", None, float),
+ ("featuresCol", "features column name.", "'features'", str),
+ ("labelCol", "label column name.", "'label'", str),
+ ("predictionCol", "prediction column name.", "'prediction'", str),
("probabilityCol", "Column name for predicted class conditional probabilities. " +
"Note: Not all models output well-calibrated probability estimates! These probabilities " +
- "should be treated as confidences, not precise probabilities.", "'probability'"),
- ("rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", "'rawPrediction'"),
- ("inputCol", "input column name.", None),
- ("inputCols", "input column names.", None),
- ("outputCol", "output column name.", "self.uid + '__output'"),
- ("numFeatures", "number of features.", None),
+ "should be treated as confidences, not precise probabilities.", "'probability'", str),
+ ("rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", "'rawPrediction'",
+ str),
+ ("inputCol", "input column name.", None, str),
+ ("inputCols", "input column names.", None, None),
+ ("outputCol", "output column name.", "self.uid + '__output'", str),
+ ("numFeatures", "number of features.", None, int),
("checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). " +
- "E.g. 10 means that the cache will get checkpointed every 10 iterations.", None),
- ("seed", "random seed.", "hash(type(self).__name__)"),
- ("tol", "the convergence tolerance for iterative algorithms.", None),
- ("stepSize", "Step size to be used for each iteration of optimization.", None),
+ "E.g. 10 means that the cache will get checkpointed every 10 iterations.", None, int),
+ ("seed", "random seed.", "hash(type(self).__name__)", int),
+ ("tol", "the convergence tolerance for iterative algorithms.", None, float),
+ ("stepSize", "Step size to be used for each iteration of optimization.", None, float),
("handleInvalid", "how to handle invalid entries. Options are skip (which will filter " +
"out rows with bad values), or error (which will throw an errror). More options may be " +
- "added later.", None),
+ "added later.", None, str),
("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, " +
- "the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", "0.0"),
- ("fitIntercept", "whether to fit an intercept term.", "True"),
+ "the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", "0.0", float),
+ ("fitIntercept", "whether to fit an intercept term.", "True", bool),
("standardization", "whether to standardize the training features before fitting the " +
- "model.", "True"),
+ "model.", "True", bool),
("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. The class with largest value p/t is predicted, where p is the original " +
- "probability of that class and t is the class' threshold.", None),
+ "probability of that class and t is the class' threshold.", None, None),
("weightCol", "weight column name. If this is not set or empty, we treat " +
- "all instance weights as 1.0.", None),
+ "all instance weights as 1.0.", None, str),
("solver", "the solver algorithm for optimization. If this is not set or empty, " +
- "default value is 'auto'.", "'auto'")]
+ "default value is 'auto'.", "'auto'", str)]
code = []
- for name, doc, defaultValueStr in shared:
- param_code = _gen_param_header(name, doc, defaultValueStr)
+ for name, doc, defaultValueStr, expectedType in shared:
+ param_code = _gen_param_header(name, doc, defaultValueStr, expectedType)
code.append(param_code + "\n" + _gen_param_code(name, doc, defaultValueStr))
decisionTreeParams = [
diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py
index 4d96080150..23f9431484 100644
--- a/python/pyspark/ml/param/shared.py
+++ b/python/pyspark/ml/param/shared.py
@@ -26,18 +26,18 @@ class HasMaxIter(Params):
"""
# a placeholder to make it appear in the generated doc
- maxIter = Param(Params._dummy(), "maxIter", "max number of iterations (>= 0).")
+ maxIter = Param(Params._dummy(), "maxIter", "max number of iterations (>= 0).", int)
def __init__(self):
super(HasMaxIter, self).__init__()
#: param for max number of iterations (>= 0).
- self.maxIter = Param(self, "maxIter", "max number of iterations (>= 0).")
+ self.maxIter = Param(self, "maxIter", "max number of iterations (>= 0).", int)
def setMaxIter(self, value):
"""
Sets the value of :py:attr:`maxIter`.
"""
- self._paramMap[self.maxIter] = value
+ self._set(maxIter=value)
return self
def getMaxIter(self):
@@ -53,18 +53,18 @@ class HasRegParam(Params):
"""
# a placeholder to make it appear in the generated doc
- regParam = Param(Params._dummy(), "regParam", "regularization parameter (>= 0).")
+ regParam = Param(Params._dummy(), "regParam", "regularization parameter (>= 0).", float)
def __init__(self):
super(HasRegParam, self).__init__()
#: param for regularization parameter (>= 0).
- self.regParam = Param(self, "regParam", "regularization parameter (>= 0).")
+ self.regParam = Param(self, "regParam", "regularization parameter (>= 0).", float)
def setRegParam(self, value):
"""
Sets the value of :py:attr:`regParam`.
"""
- self._paramMap[self.regParam] = value
+ self._set(regParam=value)
return self
def getRegParam(self):
@@ -80,19 +80,19 @@ class HasFeaturesCol(Params):
"""
# a placeholder to make it appear in the generated doc
- featuresCol = Param(Params._dummy(), "featuresCol", "features column name.")
+ featuresCol = Param(Params._dummy(), "featuresCol", "features column name.", str)
def __init__(self):
super(HasFeaturesCol, self).__init__()
#: param for features column name.
- self.featuresCol = Param(self, "featuresCol", "features column name.")
+ self.featuresCol = Param(self, "featuresCol", "features column name.", str)
self._setDefault(featuresCol='features')
def setFeaturesCol(self, value):
"""
Sets the value of :py:attr:`featuresCol`.
"""
- self._paramMap[self.featuresCol] = value
+ self._set(featuresCol=value)
return self
def getFeaturesCol(self):
@@ -108,19 +108,19 @@ class HasLabelCol(Params):
"""
# a placeholder to make it appear in the generated doc
- labelCol = Param(Params._dummy(), "labelCol", "label column name.")
+ labelCol = Param(Params._dummy(), "labelCol", "label column name.", str)
def __init__(self):
super(HasLabelCol, self).__init__()
#: param for label column name.
- self.labelCol = Param(self, "labelCol", "label column name.")
+ self.labelCol = Param(self, "labelCol", "label column name.", str)
self._setDefault(labelCol='label')
def setLabelCol(self, value):
"""
Sets the value of :py:attr:`labelCol`.
"""
- self._paramMap[self.labelCol] = value
+ self._set(labelCol=value)
return self
def getLabelCol(self):
@@ -136,19 +136,19 @@ class HasPredictionCol(Params):
"""
# a placeholder to make it appear in the generated doc
- predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name.")
+ predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name.", str)
def __init__(self):
super(HasPredictionCol, self).__init__()
#: param for prediction column name.
- self.predictionCol = Param(self, "predictionCol", "prediction column name.")
+ self.predictionCol = Param(self, "predictionCol", "prediction column name.", str)
self._setDefault(predictionCol='prediction')
def setPredictionCol(self, value):
"""
Sets the value of :py:attr:`predictionCol`.
"""
- self._paramMap[self.predictionCol] = value
+ self._set(predictionCol=value)
return self
def getPredictionCol(self):
@@ -164,19 +164,19 @@ class HasProbabilityCol(Params):
"""
# a placeholder to make it appear in the generated doc
- probabilityCol = Param(Params._dummy(), "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.")
+ probabilityCol = Param(Params._dummy(), "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.", str)
def __init__(self):
super(HasProbabilityCol, self).__init__()
#: param for Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.
- self.probabilityCol = Param(self, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.")
+ self.probabilityCol = Param(self, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.", str)
self._setDefault(probabilityCol='probability')
def setProbabilityCol(self, value):
"""
Sets the value of :py:attr:`probabilityCol`.
"""
- self._paramMap[self.probabilityCol] = value
+ self._set(probabilityCol=value)
return self
def getProbabilityCol(self):
@@ -192,19 +192,19 @@ class HasRawPredictionCol(Params):
"""
# a placeholder to make it appear in the generated doc
- rawPredictionCol = Param(Params._dummy(), "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.")
+ rawPredictionCol = Param(Params._dummy(), "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", str)
def __init__(self):
super(HasRawPredictionCol, self).__init__()
#: param for raw prediction (a.k.a. confidence) column name.
- self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.")
+ self.rawPredictionCol = Param(self, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name.", str)
self._setDefault(rawPredictionCol='rawPrediction')
def setRawPredictionCol(self, value):
"""
Sets the value of :py:attr:`rawPredictionCol`.
"""
- self._paramMap[self.rawPredictionCol] = value
+ self._set(rawPredictionCol=value)
return self
def getRawPredictionCol(self):
@@ -220,18 +220,18 @@ class HasInputCol(Params):
"""
# a placeholder to make it appear in the generated doc
- inputCol = Param(Params._dummy(), "inputCol", "input column name.")
+ inputCol = Param(Params._dummy(), "inputCol", "input column name.", str)
def __init__(self):
super(HasInputCol, self).__init__()
#: param for input column name.
- self.inputCol = Param(self, "inputCol", "input column name.")
+ self.inputCol = Param(self, "inputCol", "input column name.", str)
def setInputCol(self, value):
"""
Sets the value of :py:attr:`inputCol`.
"""
- self._paramMap[self.inputCol] = value
+ self._set(inputCol=value)
return self
def getInputCol(self):
@@ -247,18 +247,18 @@ class HasInputCols(Params):
"""
# a placeholder to make it appear in the generated doc
- inputCols = Param(Params._dummy(), "inputCols", "input column names.")
+ inputCols = Param(Params._dummy(), "inputCols", "input column names.", None)
def __init__(self):
super(HasInputCols, self).__init__()
#: param for input column names.
- self.inputCols = Param(self, "inputCols", "input column names.")
+ self.inputCols = Param(self, "inputCols", "input column names.", None)
def setInputCols(self, value):
"""
Sets the value of :py:attr:`inputCols`.
"""
- self._paramMap[self.inputCols] = value
+ self._set(inputCols=value)
return self
def getInputCols(self):
@@ -274,19 +274,19 @@ class HasOutputCol(Params):
"""
# a placeholder to make it appear in the generated doc
- outputCol = Param(Params._dummy(), "outputCol", "output column name.")
+ outputCol = Param(Params._dummy(), "outputCol", "output column name.", str)
def __init__(self):
super(HasOutputCol, self).__init__()
#: param for output column name.
- self.outputCol = Param(self, "outputCol", "output column name.")
+ self.outputCol = Param(self, "outputCol", "output column name.", str)
self._setDefault(outputCol=self.uid + '__output')
def setOutputCol(self, value):
"""
Sets the value of :py:attr:`outputCol`.
"""
- self._paramMap[self.outputCol] = value
+ self._set(outputCol=value)
return self
def getOutputCol(self):
@@ -302,18 +302,18 @@ class HasNumFeatures(Params):
"""
# a placeholder to make it appear in the generated doc
- numFeatures = Param(Params._dummy(), "numFeatures", "number of features.")
+ numFeatures = Param(Params._dummy(), "numFeatures", "number of features.", int)
def __init__(self):
super(HasNumFeatures, self).__init__()
#: param for number of features.
- self.numFeatures = Param(self, "numFeatures", "number of features.")
+ self.numFeatures = Param(self, "numFeatures", "number of features.", int)
def setNumFeatures(self, value):
"""
Sets the value of :py:attr:`numFeatures`.
"""
- self._paramMap[self.numFeatures] = value
+ self._set(numFeatures=value)
return self
def getNumFeatures(self):
@@ -329,18 +329,18 @@ class HasCheckpointInterval(Params):
"""
# a placeholder to make it appear in the generated doc
- checkpointInterval = Param(Params._dummy(), "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.")
+ checkpointInterval = Param(Params._dummy(), "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.", int)
def __init__(self):
super(HasCheckpointInterval, self).__init__()
#: param for set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
- self.checkpointInterval = Param(self, "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.")
+ self.checkpointInterval = Param(self, "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.", int)
def setCheckpointInterval(self, value):
"""
Sets the value of :py:attr:`checkpointInterval`.
"""
- self._paramMap[self.checkpointInterval] = value
+ self._set(checkpointInterval=value)
return self
def getCheckpointInterval(self):
@@ -356,19 +356,19 @@ class HasSeed(Params):
"""
# a placeholder to make it appear in the generated doc
- seed = Param(Params._dummy(), "seed", "random seed.")
+ seed = Param(Params._dummy(), "seed", "random seed.", int)
def __init__(self):
super(HasSeed, self).__init__()
#: param for random seed.
- self.seed = Param(self, "seed", "random seed.")
+ self.seed = Param(self, "seed", "random seed.", int)
self._setDefault(seed=hash(type(self).__name__))
def setSeed(self, value):
"""
Sets the value of :py:attr:`seed`.
"""
- self._paramMap[self.seed] = value
+ self._set(seed=value)
return self
def getSeed(self):
@@ -384,18 +384,18 @@ class HasTol(Params):
"""
# a placeholder to make it appear in the generated doc
- tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms.")
+ tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms.", float)
def __init__(self):
super(HasTol, self).__init__()
#: param for the convergence tolerance for iterative algorithms.
- self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms.")
+ self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms.", float)
def setTol(self, value):
"""
Sets the value of :py:attr:`tol`.
"""
- self._paramMap[self.tol] = value
+ self._set(tol=value)
return self
def getTol(self):
@@ -411,18 +411,18 @@ class HasStepSize(Params):
"""
# a placeholder to make it appear in the generated doc
- stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.")
+ stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.", float)
def __init__(self):
super(HasStepSize, self).__init__()
#: param for Step size to be used for each iteration of optimization.
- self.stepSize = Param(self, "stepSize", "Step size to be used for each iteration of optimization.")
+ self.stepSize = Param(self, "stepSize", "Step size to be used for each iteration of optimization.", float)
def setStepSize(self, value):
"""
Sets the value of :py:attr:`stepSize`.
"""
- self._paramMap[self.stepSize] = value
+ self._set(stepSize=value)
return self
def getStepSize(self):
@@ -438,18 +438,18 @@ class HasHandleInvalid(Params):
"""
# a placeholder to make it appear in the generated doc
- handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.")
+ handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", str)
def __init__(self):
super(HasHandleInvalid, self).__init__()
#: param for how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.
- self.handleInvalid = Param(self, "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.")
+ self.handleInvalid = Param(self, "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", str)
def setHandleInvalid(self, value):
"""
Sets the value of :py:attr:`handleInvalid`.
"""
- self._paramMap[self.handleInvalid] = value
+ self._set(handleInvalid=value)
return self
def getHandleInvalid(self):
@@ -465,19 +465,19 @@ class HasElasticNetParam(Params):
"""
# a placeholder to make it appear in the generated doc
- elasticNetParam = Param(Params._dummy(), "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.")
+ elasticNetParam = Param(Params._dummy(), "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", float)
def __init__(self):
super(HasElasticNetParam, self).__init__()
#: param for the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.
- self.elasticNetParam = Param(self, "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.")
+ self.elasticNetParam = Param(self, "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", float)
self._setDefault(elasticNetParam=0.0)
def setElasticNetParam(self, value):
"""
Sets the value of :py:attr:`elasticNetParam`.
"""
- self._paramMap[self.elasticNetParam] = value
+ self._set(elasticNetParam=value)
return self
def getElasticNetParam(self):
@@ -493,19 +493,19 @@ class HasFitIntercept(Params):
"""
# a placeholder to make it appear in the generated doc
- fitIntercept = Param(Params._dummy(), "fitIntercept", "whether to fit an intercept term.")
+ fitIntercept = Param(Params._dummy(), "fitIntercept", "whether to fit an intercept term.", bool)
def __init__(self):
super(HasFitIntercept, self).__init__()
#: param for whether to fit an intercept term.
- self.fitIntercept = Param(self, "fitIntercept", "whether to fit an intercept term.")
+ self.fitIntercept = Param(self, "fitIntercept", "whether to fit an intercept term.", bool)
self._setDefault(fitIntercept=True)
def setFitIntercept(self, value):
"""
Sets the value of :py:attr:`fitIntercept`.
"""
- self._paramMap[self.fitIntercept] = value
+ self._set(fitIntercept=value)
return self
def getFitIntercept(self):
@@ -521,19 +521,19 @@ class HasStandardization(Params):
"""
# a placeholder to make it appear in the generated doc
- standardization = Param(Params._dummy(), "standardization", "whether to standardize the training features before fitting the model.")
+ standardization = Param(Params._dummy(), "standardization", "whether to standardize the training features before fitting the model.", bool)
def __init__(self):
super(HasStandardization, self).__init__()
#: param for whether to standardize the training features before fitting the model.
- self.standardization = Param(self, "standardization", "whether to standardize the training features before fitting the model.")
+ self.standardization = Param(self, "standardization", "whether to standardize the training features before fitting the model.", bool)
self._setDefault(standardization=True)
def setStandardization(self, value):
"""
Sets the value of :py:attr:`standardization`.
"""
- self._paramMap[self.standardization] = value
+ self._set(standardization=value)
return self
def getStandardization(self):
@@ -549,18 +549,18 @@ class HasThresholds(Params):
"""
# a placeholder to make it appear in the generated doc
- thresholds = Param(Params._dummy(), "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. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.")
+ thresholds = Param(Params._dummy(), "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. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.", None)
def __init__(self):
super(HasThresholds, self).__init__()
#: param for 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. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.
- self.thresholds = Param(self, "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. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.")
+ self.thresholds = Param(self, "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. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.", None)
def setThresholds(self, value):
"""
Sets the value of :py:attr:`thresholds`.
"""
- self._paramMap[self.thresholds] = value
+ self._set(thresholds=value)
return self
def getThresholds(self):
@@ -576,18 +576,18 @@ class HasWeightCol(Params):
"""
# a placeholder to make it appear in the generated doc
- weightCol = Param(Params._dummy(), "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.")
+ weightCol = Param(Params._dummy(), "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.", str)
def __init__(self):
super(HasWeightCol, self).__init__()
#: param for weight column name. If this is not set or empty, we treat all instance weights as 1.0.
- self.weightCol = Param(self, "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.")
+ self.weightCol = Param(self, "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.", str)
def setWeightCol(self, value):
"""
Sets the value of :py:attr:`weightCol`.
"""
- self._paramMap[self.weightCol] = value
+ self._set(weightCol=value)
return self
def getWeightCol(self):
@@ -603,19 +603,19 @@ class HasSolver(Params):
"""
# a placeholder to make it appear in the generated doc
- solver = Param(Params._dummy(), "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.")
+ solver = Param(Params._dummy(), "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.", str)
def __init__(self):
super(HasSolver, self).__init__()
#: param for the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.
- self.solver = Param(self, "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.")
+ self.solver = Param(self, "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.", str)
self._setDefault(solver='auto')
def setSolver(self, value):
"""
Sets the value of :py:attr:`solver`.
"""
- self._paramMap[self.solver] = value
+ self._set(solver=value)
return self
def getSolver(self):
@@ -658,7 +658,7 @@ class DecisionTreeParams(Params):
"""
Sets the value of :py:attr:`maxDepth`.
"""
- self._paramMap[self.maxDepth] = value
+ self._set(maxDepth=value)
return self
def getMaxDepth(self):
@@ -671,7 +671,7 @@ class DecisionTreeParams(Params):
"""
Sets the value of :py:attr:`maxBins`.
"""
- self._paramMap[self.maxBins] = value
+ self._set(maxBins=value)
return self
def getMaxBins(self):
@@ -684,7 +684,7 @@ class DecisionTreeParams(Params):
"""
Sets the value of :py:attr:`minInstancesPerNode`.
"""
- self._paramMap[self.minInstancesPerNode] = value
+ self._set(minInstancesPerNode=value)
return self
def getMinInstancesPerNode(self):
@@ -697,7 +697,7 @@ class DecisionTreeParams(Params):
"""
Sets the value of :py:attr:`minInfoGain`.
"""
- self._paramMap[self.minInfoGain] = value
+ self._set(minInfoGain=value)
return self
def getMinInfoGain(self):
@@ -710,7 +710,7 @@ class DecisionTreeParams(Params):
"""
Sets the value of :py:attr:`maxMemoryInMB`.
"""
- self._paramMap[self.maxMemoryInMB] = value
+ self._set(maxMemoryInMB=value)
return self
def getMaxMemoryInMB(self):
@@ -723,7 +723,7 @@ class DecisionTreeParams(Params):
"""
Sets the value of :py:attr:`cacheNodeIds`.
"""
- self._paramMap[self.cacheNodeIds] = value
+ self._set(cacheNodeIds=value)
return self
def getCacheNodeIds(self):
diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py
index 7a16cf52cc..4eb17bfdcc 100644
--- a/python/pyspark/ml/tests.py
+++ b/python/pyspark/ml/tests.py
@@ -37,6 +37,7 @@ else:
from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase
from pyspark.sql import DataFrame, SQLContext, Row
from pyspark.sql.functions import rand
+from pyspark.ml.classification import LogisticRegression
from pyspark.ml.evaluation import RegressionEvaluator
from pyspark.ml.param import Param, Params
from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed
@@ -92,6 +93,27 @@ class MockModel(MockTransformer, Model, HasFake):
pass
+class ParamTypeConversionTests(PySparkTestCase):
+ """
+ Test that param type conversion happens.
+ """
+
+ def test_int_to_float(self):
+ from pyspark.mllib.linalg import Vectors
+ df = self.sc.parallelize([
+ Row(label=1.0, weight=2.0, features=Vectors.dense(1.0))]).toDF()
+ lr = LogisticRegression(elasticNetParam=0)
+ lr.fit(df)
+ lr.setElasticNetParam(0)
+ lr.fit(df)
+
+ def test_invalid_to_float(self):
+ from pyspark.mllib.linalg import Vectors
+ self.assertRaises(Exception, lambda: LogisticRegression(elasticNetParam="happy"))
+ lr = LogisticRegression(elasticNetParam=0)
+ self.assertRaises(Exception, lambda: lr.setElasticNetParam("panda"))
+
+
class PipelineTests(PySparkTestCase):
def test_pipeline(self):