-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21027][ML][PYTHON] Added tunable parallelism to one vs. rest in both Scala mllib and Pyspark #18281
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-21027][ML][PYTHON] Added tunable parallelism to one vs. rest in both Scala mllib and Pyspark #18281
Changes from 1 commit
b69f201
e750d3e
81d458b
2133378
c59b1d8
5f635a2
4431ffc
a841b3e
a95a8af
d45bc23
30ac62d
cc634d2
ce14172
1c9de16
9f34404
585a3f8
f65381a
2a335fe
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
…st classification. Added a parallelism parameter to the scala implementation of one vs. rest for python persistence but have not yet used it to tune the scala parallelism implementation.
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,7 @@ | |
| # | ||
|
|
||
| import operator | ||
| from multiprocessing.pool import ThreadPool | ||
|
|
||
| from pyspark import since, keyword_only | ||
| from pyspark.ml import Estimator, Model | ||
|
|
@@ -1510,21 +1511,25 @@ class OneVsRest(Estimator, OneVsRestParams, MLReadable, MLWritable): | |
|
|
||
| .. versionadded:: 2.0.0 | ||
| """ | ||
| parallelism = Param(Params._dummy(), "parallelism", | ||
|
||
| "Number of models to fit in parallel", | ||
|
||
| typeConverter=TypeConverters.toInt) | ||
|
|
||
| @keyword_only | ||
| def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", | ||
| classifier=None): | ||
| classifier=None, parallelism=8): | ||
| """ | ||
| __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ | ||
| classifier=None) | ||
|
||
| """ | ||
| super(OneVsRest, self).__init__() | ||
| self._setDefault(parallelism=8) | ||
| kwargs = self._input_kwargs | ||
| self._set(**kwargs) | ||
|
|
||
| @keyword_only | ||
| @since("2.0.0") | ||
| def setParams(self, featuresCol=None, labelCol=None, predictionCol=None, classifier=None): | ||
| def setParams(self, featuresCol=None, labelCol=None, predictionCol=None, classifier=None, parallelism=None): | ||
| """ | ||
| setParams(self, featuresCol=None, labelCol=None, predictionCol=None, classifier=None): | ||
|
||
| Sets params for OneVsRest. | ||
|
|
@@ -1561,13 +1566,28 @@ def trainSingleClass(index): | |
| return classifier.fit(trainingDataset, paramMap) | ||
|
|
||
| # TODO: Parallel training for all classes. | ||
| models = [trainSingleClass(i) for i in range(numClasses)] | ||
| pool = ThreadPool(processes=self.getParallelism()) | ||
|
||
|
|
||
| models = pool.map(trainSingleClass, range(numClasses)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So if the training of the models takes different times and there is a relatively large number of models we might end up blocking a worker here since the map effectively splits in advance. What do you think of providing a chunksize hint or using |
||
| #models = [trainSingleClass(i) for i in range(numClasses)] | ||
|
|
||
| if handlePersistence: | ||
| multiclassLabeled.unpersist() | ||
|
|
||
| return self._copyValues(OneVsRestModel(models=models)) | ||
|
|
||
| def setParallelism(self, value): | ||
|
||
| """ | ||
| Sets the value of :py:attr:`parallelism`. | ||
| """ | ||
| return self._set(parallelism=value) | ||
|
|
||
| def getParallelism(self): | ||
| """ | ||
| Gets the value of parallelism or its default value. | ||
| """ | ||
| return self.getOrDefault(self.parallelism) | ||
|
|
||
| @since("2.0.0") | ||
| def copy(self, extra=None): | ||
| """ | ||
|
|
@@ -1611,8 +1631,9 @@ def _from_java(cls, java_stage): | |
| labelCol = java_stage.getLabelCol() | ||
| predictionCol = java_stage.getPredictionCol() | ||
| classifier = JavaParams._from_java(java_stage.getClassifier()) | ||
| parallelism = java_stage.getParallelism() | ||
| py_stage = cls(featuresCol=featuresCol, labelCol=labelCol, predictionCol=predictionCol, | ||
| classifier=classifier) | ||
| classifier=classifier, parallelism=parallelism) | ||
| py_stage._resetUid(java_stage.uid()) | ||
| return py_stage | ||
|
|
||
|
|
@@ -1625,12 +1646,12 @@ def _to_java(self): | |
| _java_obj = JavaParams._new_java_obj("org.apache.spark.ml.classification.OneVsRest", | ||
| self.uid) | ||
| _java_obj.setClassifier(self.getClassifier()._to_java()) | ||
| _java_obj.setParallelism(self.getParallelism()) | ||
| _java_obj.setFeaturesCol(self.getFeaturesCol()) | ||
| _java_obj.setLabelCol(self.getLabelCol()) | ||
| _java_obj.setPredictionCol(self.getPredictionCol()) | ||
| return _java_obj | ||
|
|
||
|
|
||
| class OneVsRestModel(Model, OneVsRestParams, MLReadable, MLWritable): | ||
| """ | ||
| .. note:: Experimental | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -951,7 +951,7 @@ def test_onevsrest(self): | |
| (2.0, Vectors.dense(0.5, 0.5))] * 10, | ||
| ["label", "features"]) | ||
| lr = LogisticRegression(maxIter=5, regParam=0.01) | ||
| ovr = OneVsRest(classifier=lr) | ||
| ovr = OneVsRest(classifier=lr, parallelism=8) | ||
|
||
| model = ovr.fit(df) | ||
| ovrPath = temp_path + "/ovr" | ||
| ovr.save(ovrPath) | ||
|
|
@@ -1215,7 +1215,7 @@ def test_copy(self): | |
| (2.0, Vectors.dense(0.5, 0.5))], | ||
| ["label", "features"]) | ||
| lr = LogisticRegression(maxIter=5, regParam=0.01) | ||
| ovr = OneVsRest(classifier=lr) | ||
| ovr = OneVsRest(classifier=lr, parallelism=1) | ||
|
||
| ovr1 = ovr.copy({lr.maxIter: 10}) | ||
| self.assertEqual(ovr.getClassifier().getMaxIter(), 5) | ||
| self.assertEqual(ovr1.getClassifier().getMaxIter(), 10) | ||
|
|
@@ -1229,11 +1229,37 @@ def test_output_columns(self): | |
| (2.0, Vectors.dense(0.5, 0.5))], | ||
| ["label", "features"]) | ||
| lr = LogisticRegression(maxIter=5, regParam=0.01) | ||
| ovr = OneVsRest(classifier=lr) | ||
| ovr = OneVsRest(classifier=lr, parallelism=1) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same as above |
||
| model = ovr.fit(df) | ||
| output = model.transform(df) | ||
| self.assertEqual(output.columns, ["label", "features", "prediction"]) | ||
|
|
||
| class ParOneVsRestTests(SparkSessionTestCase): | ||
|
|
||
| def test_copy(self): | ||
|
||
| df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), | ||
| (1.0, Vectors.sparse(2, [], [])), | ||
| (2.0, Vectors.dense(0.5, 0.5))], | ||
| ["label", "features"]) | ||
| lr = LogisticRegression(maxIter=5, regParam=0.01) | ||
| ovr = OneVsRest(classifier=lr, parallelism=8) | ||
| ovr1 = ovr.copy({lr.maxIter: 10}) | ||
| self.assertEqual(ovr.getClassifier().getMaxIter(), 5) | ||
| self.assertEqual(ovr1.getClassifier().getMaxIter(), 10) | ||
| model = ovr.fit(df) | ||
| model1 = model.copy({model.predictionCol: "indexed"}) | ||
| self.assertEqual(model1.getPredictionCol(), "indexed") | ||
|
|
||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Similar comment, can we add a test to make sure that we are actually training in parallel? This is perhaps especially important in Python because I could see us accidentally blocking on something unexpected. |
||
| def test_output_columns(self): | ||
|
||
| df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), | ||
| (1.0, Vectors.sparse(2, [], [])), | ||
| (2.0, Vectors.dense(0.5, 0.5))], | ||
| ["label", "features"]) | ||
| lr = LogisticRegression(maxIter=5, regParam=0.01) | ||
| ovr = OneVsRest(classifier=lr, parallelism=8) | ||
| model = ovr.fit(df) | ||
| output = model.transform(df) | ||
| self.assertEqual(output.columns, ["label", "features", "prediction"]) | ||
|
|
||
| class HashingTFTest(SparkSessionTestCase): | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
next release will be 2.3