Skip to content

Commit

Permalink
update ml/tests.py to test copy params
Browse files Browse the repository at this point in the history
  • Loading branch information
mengxr committed May 15, 2015
1 parent 7e0d27f commit c7d84ef
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 7 deletions.
8 changes: 4 additions & 4 deletions python/pyspark/ml/param/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -70,9 +70,9 @@ class Params(Identifiable):
@property
def params(self):
"""
Returns all params ordered by name. The default implementation
uses :py:func:`dir` to get all attributes of type
:py:class:`Param`.
Returns all params as a namedtuple, ordered by name. The
default implementation uses :py:func:`dir` to get all
attributes of type :py:class:`Param`.
"""
if self._params is None:
self._params = list(filter(lambda attr: isinstance(attr, Param),
Expand Down Expand Up @@ -243,5 +243,5 @@ def _copyValues(self, to, extra={}):
paramMap = self.extractParamMap(extra)
for p in self.params:
if p in paramMap and to.hasParam(p.name):
to._set((p.name, paramMap[p]))
to._set(**{p.name: paramMap[p]})
return to
11 changes: 8 additions & 3 deletions python/pyspark/ml/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
from pyspark.sql import DataFrame
from pyspark.ml.param import Param, Params
from pyspark.ml.param.shared import HasMaxIter, HasInputCol
from pyspark.ml.pipeline import Estimator, Model, Pipeline, Transformer
from pyspark.ml import Estimator, Model, Pipeline, Transformer


class MockDataset(DataFrame):
Expand All @@ -49,6 +49,9 @@ def __init__(self):
super(HasFake, self).__init__()
self.fake = Param(self, "fake", "fake param")

def getFake(self):
return self.getOrDefault(self.fake)


class MockTransformer(Transformer, HasFake):

Expand All @@ -71,6 +74,7 @@ def __init__(self):
def _fit(self, dataset):
self.dataset_index = dataset.index
model = MockModel()
self._copyValues(model)
return model


Expand All @@ -86,12 +90,13 @@ def test_pipeline(self):
transformer1 = MockTransformer()
estimator2 = MockEstimator()
transformer3 = MockTransformer()
pipeline = Pipeline() \
.setStages([estimator0, transformer1, estimator2, transformer3])
pipeline = Pipeline(stages=[estimator0, transformer1, estimator2, transformer3])
pipeline_model = pipeline.fit(dataset, {estimator0.fake: 0, transformer1.fake: 1})
model0, transformer1, model2, transformer3 = pipeline_model.stages
self.assertEqual(0, model0.dataset_index)
self.assertEqual(0, model0.getFake())
self.assertEqual(1, transformer1.dataset_index)
self.assertEqual(1, transformer1.getFake())
self.assertEqual(2, dataset.index)
self.assertIsNone(model2.dataset_index, "The last model shouldn't be called in fit.")
self.assertIsNone(transformer3.dataset_index,
Expand Down

0 comments on commit c7d84ef

Please sign in to comment.