aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/ml
diff options
context:
space:
mode:
authorMechCoder <manojkumarsivaraj334@gmail.com>2015-08-14 12:46:05 -0700
committerXiangrui Meng <meng@databricks.com>2015-08-14 12:46:05 -0700
commitffa05c84fe75663fc33f3d954d1cb1e084ab3280 (patch)
treeeb5599d7c52bf7ddcd72d340952bca83e169c2d0 /python/pyspark/ml
parentece00566e4d5f38585f2810bef38e526cae7d25e (diff)
downloadspark-ffa05c84fe75663fc33f3d954d1cb1e084ab3280.tar.gz
spark-ffa05c84fe75663fc33f3d954d1cb1e084ab3280.tar.bz2
spark-ffa05c84fe75663fc33f3d954d1cb1e084ab3280.zip
[SPARK-9828] [PYSPARK] Mutable values should not be default arguments
Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #8110 from MechCoder/spark-9828.
Diffstat (limited to 'python/pyspark/ml')
-rw-r--r--python/pyspark/ml/evaluation.py4
-rw-r--r--python/pyspark/ml/param/__init__.py26
-rw-r--r--python/pyspark/ml/pipeline.py4
-rw-r--r--python/pyspark/ml/tuning.py8
4 files changed, 28 insertions, 14 deletions
diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py
index 2734092575..e23ce053ba 100644
--- a/python/pyspark/ml/evaluation.py
+++ b/python/pyspark/ml/evaluation.py
@@ -46,7 +46,7 @@ class Evaluator(Params):
"""
raise NotImplementedError()
- def evaluate(self, dataset, params={}):
+ def evaluate(self, dataset, params=None):
"""
Evaluates the output with optional parameters.
@@ -56,6 +56,8 @@ class Evaluator(Params):
params
:return: metric
"""
+ if params is None:
+ params = dict()
if isinstance(params, dict):
if params:
return self.copy(params)._evaluate(dataset)
diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py
index 7845536161..eeeac49b21 100644
--- a/python/pyspark/ml/param/__init__.py
+++ b/python/pyspark/ml/param/__init__.py
@@ -60,14 +60,16 @@ class Params(Identifiable):
__metaclass__ = ABCMeta
- #: internal param map for user-supplied values param map
- _paramMap = {}
+ def __init__(self):
+ super(Params, self).__init__()
+ #: internal param map for user-supplied values param map
+ self._paramMap = {}
- #: internal param map for default values
- _defaultParamMap = {}
+ #: internal param map for default values
+ self._defaultParamMap = {}
- #: value returned by :py:func:`params`
- _params = None
+ #: value returned by :py:func:`params`
+ self._params = None
@property
def params(self):
@@ -155,7 +157,7 @@ class Params(Identifiable):
else:
return self._defaultParamMap[param]
- def extractParamMap(self, extra={}):
+ def extractParamMap(self, extra=None):
"""
Extracts the embedded default param values and user-supplied
values, and then merges them with extra values from input into
@@ -165,12 +167,14 @@ class Params(Identifiable):
:param extra: extra param values
:return: merged param map
"""
+ if extra is None:
+ extra = dict()
paramMap = self._defaultParamMap.copy()
paramMap.update(self._paramMap)
paramMap.update(extra)
return paramMap
- def copy(self, extra={}):
+ def copy(self, extra=None):
"""
Creates a copy of this instance with the same uid and some
extra params. The default implementation creates a
@@ -181,6 +185,8 @@ class Params(Identifiable):
:param extra: Extra parameters to copy to the new instance
:return: Copy of this instance
"""
+ if extra is None:
+ extra = dict()
that = copy.copy(self)
that._paramMap = self.extractParamMap(extra)
return that
@@ -233,7 +239,7 @@ class Params(Identifiable):
self._defaultParamMap[getattr(self, param)] = value
return self
- def _copyValues(self, to, extra={}):
+ def _copyValues(self, to, extra=None):
"""
Copies param values from this instance to another instance for
params shared by them.
@@ -241,6 +247,8 @@ class Params(Identifiable):
:param extra: extra params to be copied
:return: the target instance with param values copied
"""
+ if extra is None:
+ extra = dict()
paramMap = self.extractParamMap(extra)
for p in self.params:
if p in paramMap and to.hasParam(p.name):
diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py
index 9889f56cac..13cf2b0f7b 100644
--- a/python/pyspark/ml/pipeline.py
+++ b/python/pyspark/ml/pipeline.py
@@ -141,7 +141,7 @@ class Pipeline(Estimator):
@keyword_only
def __init__(self, stages=None):
"""
- __init__(self, stages=[])
+ __init__(self, stages=None)
"""
if stages is None:
stages = []
@@ -170,7 +170,7 @@ class Pipeline(Estimator):
@keyword_only
def setParams(self, stages=None):
"""
- setParams(self, stages=[])
+ setParams(self, stages=None)
Sets params for Pipeline.
"""
if stages is None:
diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py
index 0bf988fd72..dcfee6a317 100644
--- a/python/pyspark/ml/tuning.py
+++ b/python/pyspark/ml/tuning.py
@@ -227,7 +227,9 @@ class CrossValidator(Estimator):
bestModel = est.fit(dataset, epm[bestIndex])
return CrossValidatorModel(bestModel)
- def copy(self, extra={}):
+ def copy(self, extra=None):
+ if extra is None:
+ extra = dict()
newCV = Params.copy(self, extra)
if self.isSet(self.estimator):
newCV.setEstimator(self.getEstimator().copy(extra))
@@ -250,7 +252,7 @@ class CrossValidatorModel(Model):
def _transform(self, dataset):
return self.bestModel.transform(dataset)
- def copy(self, extra={}):
+ def copy(self, extra=None):
"""
Creates a copy of this instance with a randomly generated uid
and some extra params. This copies the underlying bestModel,
@@ -259,6 +261,8 @@ class CrossValidatorModel(Model):
:param extra: Extra parameters to copy to the new instance
:return: Copy of this instance
"""
+ if extra is None:
+ extra = dict()
return CrossValidatorModel(self.bestModel.copy(extra))