From 16466208da85989c1e0b47a668507328ec936f82 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 5 Dec 2017 17:56:35 +0800 Subject: [PATCH 01/11] create pr --- python/pyspark/ml/feature.py | 70 +++++++++++++++++++++++++---- python/pyspark/ml/param/__init__.py | 14 ++++++ 2 files changed, 75 insertions(+), 9 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index eb79b193103e..89523b07080c 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -317,13 +317,13 @@ class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable) @inherit_doc -class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, - JavaMLReadable, JavaMLWritable): +class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, + HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ Maps a column of continuous features to a column of feature buckets. - >>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)] - >>> df = spark.createDataFrame(values, ["values"]) + >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), (float("nan"), 1.0), (float("nan"), 0.0)] + >>> df = spark.createDataFrame(values, ["values", "numbers"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], ... inputCol="values", outputCol="buckets") >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df).collect() @@ -347,6 +347,27 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 + >>> bucketizer2 = Bucketizer(splitsArray=[[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]], + ... inputCols=["values", "numbers"], outputCols=["buckets1", "buckets2"]) + >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df).collect() + >>> len(bucketed2) + 6 + >>> bucketed2[0].buckets1 + 0.0 + >>> bucketed2[1].buckets1 + 0.0 + >>> bucketed2[2].buckets1 + 1.0 + >>> bucketed2[3].buckets1 + 2.0 + >>> bucketed2[0].buckets2 + 0.0 + >>> bucketed2[1].buckets2 + 1.0 + >>> bucketed2[2].buckets2 + 1.0 + >>> bucketed2[3].buckets2 + 2.0 .. versionadded:: 1.4.0 """ @@ -364,13 +385,28 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. " + "Options are 'skip' (filter out rows with invalid values), " + "'error' (throw an error), or 'keep' (keep invalid values in a special " + - "additional bucket).", + "additional bucket). Note that in the multiple column case, the invalid " + + "handling is applied to all columns. That said for 'error' it will throw " + + "an error if any invalids are found in any column, for 'skip' it will " + + "skip rows with any invalids in any columns, etc.", typeConverter=TypeConverters.toString) + splitsArray = Param(Params._dummy(), "splitsArray", "The array of split points for mapping " + + "continuous features into buckets for multiple columns. For each input " + + "column, with n+1 splits, there are n buckets. A bucket defined by splits " + + "x,y holds values in the range [x,y) except the last bucket, which also " + + "includes y. The splits should be of length >= 3 and strictly increasing. " + + "Values at -inf, inf must be explicitly provided to cover all Double " + + "values; otherwise, values outside the splits specified will be treated " + + "as errors.", + typeConverter=TypeConverters.toListListFloat) + @keyword_only - def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"): + def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None): """ - __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error") + __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None) """ super(Bucketizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Bucketizer", self.uid) @@ -380,9 +416,11 @@ def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="er @keyword_only @since("1.4.0") - def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"): + def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None): """ - setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error") + setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None) Sets params for this Bucketizer. """ kwargs = self._input_kwargs @@ -402,6 +440,20 @@ def getSplits(self): """ return self.getOrDefault(self.splits) + @since("2.3.0") + def setSplitsArray(self, value): + """ + Sets the value of :py:attr:`splitsArray`. + """ + return self._set(splitsArray=value) + + @since("2.3.0") + def getSplitsArray(self): + """ + Gets the array of split points or its default value. + """ + return self.getOrDefault(self.splitsArray) + @inherit_doc class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 043c25cf9feb..a6c07a2f6163 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -134,6 +134,20 @@ def toListFloat(value): return [float(v) for v in value] raise TypeError("Could not convert %s to list of floats" % value) + @staticmethod + def toListListFloat(value): + """ + Convert a value to list of list of floats, if possible. + """ + if TypeConverters._can_convert_to_list(value): + value = TypeConverters.toList(value) + if all(map(lambda v: TypeConverters._can_convert_to_list(v), value)): + ll = [] + for arr in value: + ll.append([float(v) for v in arr]) + return ll + raise TypeError("Could not convert %s to list of list of floats" % value) + @staticmethod def toListInt(value): """ From 51d5cfa0b260210c0bb2ed2ddfa286cf2f0252ba Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 5 Dec 2017 18:00:32 +0800 Subject: [PATCH 02/11] update --- python/pyspark/ml/param/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index a6c07a2f6163..3840d169bc44 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -143,8 +143,8 @@ def toListListFloat(value): value = TypeConverters.toList(value) if all(map(lambda v: TypeConverters._can_convert_to_list(v), value)): ll = [] - for arr in value: - ll.append([float(v) for v in arr]) + for v in value: + ll.append([float(i) for i in TypeConverters.toList(v)]) return ll raise TypeError("Could not convert %s to list of list of floats" % value) From 39a888f3d203809485fa3669f2c838b902b42462 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 5 Dec 2017 18:20:35 +0800 Subject: [PATCH 03/11] update style --- python/pyspark/ml/feature.py | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 89523b07080c..15506cabf53b 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -322,7 +322,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu """ Maps a column of continuous features to a column of feature buckets. - >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), (float("nan"), 1.0), (float("nan"), 0.0)] + >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), + ... (float("nan"), 1.0), (float("nan"), 0.0)] >>> df = spark.createDataFrame(values, ["values", "numbers"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], ... inputCol="values", outputCol="buckets") @@ -347,7 +348,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 - >>> bucketizer2 = Bucketizer(splitsArray=[[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]], + >>> bucketizer2 = Bucketizer(splitsArray=[[-float("inf"), 0.5, 1.4, float("inf")], + ... [-float("inf"), 0.5, float("inf")]], ... inputCols=["values", "numbers"], outputCols=["buckets1", "buckets2"]) >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df).collect() >>> len(bucketed2) @@ -384,21 +386,22 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. " + "Options are 'skip' (filter out rows with invalid values), " + - "'error' (throw an error), or 'keep' (keep invalid values in a special " + - "additional bucket). Note that in the multiple column case, the invalid " + - "handling is applied to all columns. That said for 'error' it will throw " + - "an error if any invalids are found in any column, for 'skip' it will " + - "skip rows with any invalids in any columns, etc.", + "'error' (throw an error), or 'keep' (keep invalid values in a " + + "special additional bucket). Note that in the multiple column " + + "case, the invalid handling is applied to all columns. That said " + + "for 'error' it will throw an error if any invalids are found in " + + "any column, for 'skip' it will skip rows with any invalids in " + + "any columns, etc.", typeConverter=TypeConverters.toString) splitsArray = Param(Params._dummy(), "splitsArray", "The array of split points for mapping " + "continuous features into buckets for multiple columns. For each input " + - "column, with n+1 splits, there are n buckets. A bucket defined by splits " + - "x,y holds values in the range [x,y) except the last bucket, which also " + - "includes y. The splits should be of length >= 3 and strictly increasing. " + - "Values at -inf, inf must be explicitly provided to cover all Double " + - "values; otherwise, values outside the splits specified will be treated " + - "as errors.", + "column, with n+1 splits, there are n buckets. A bucket defined by " + + "splits x,y holds values in the range [x,y) except the last bucket, " + + "which also includes y. The splits should be of length >= 3 and " + + "strictly increasing. Values at -inf, inf must be explicitly provided " + + "to cover all Double values; otherwise, values outside the splits " + + "specified will be treated as errors.", typeConverter=TypeConverters.toListListFloat) @keyword_only From 6f82831ac4d03f3923083276fd80e0f237e417d5 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 5 Dec 2017 18:47:49 +0800 Subject: [PATCH 04/11] update style 2 --- python/pyspark/ml/feature.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 15506cabf53b..8401c6c1f608 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -348,8 +348,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 - >>> bucketizer2 = Bucketizer(splitsArray=[[-float("inf"), 0.5, 1.4, float("inf")], - ... [-float("inf"), 0.5, float("inf")]], + >>> bucketizer2 = Bucketizer(splitsArray= + ... [[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]], ... inputCols=["values", "numbers"], outputCols=["buckets1", "buckets2"]) >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df).collect() >>> len(bucketed2) @@ -408,7 +408,7 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", splitsArray=None, inputCols=None, outputCols=None): """ - __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \ splitsArray=None, inputCols=None, outputCols=None) """ super(Bucketizer, self).__init__() @@ -422,7 +422,7 @@ def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="er def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", splitsArray=None, inputCols=None, outputCols=None): """ - setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \ splitsArray=None, inputCols=None, outputCols=None) Sets params for this Bucketizer. """ From 248954f88f60ffbcc4a22ecdc4e6e3983e2d6ae5 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 19 Dec 2017 11:02:39 +0800 Subject: [PATCH 05/11] update doc --- python/pyspark/ml/feature.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 8401c6c1f608..cb32a0373687 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -320,7 +320,12 @@ class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable) class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ - Maps a column of continuous features to a column of feature buckets. + Maps a column of continuous features to a column of feature buckets. Since 2.3.0, + :py:class:`Bucketizer` can map multiple columns at once by setting the :py:attr:`inputCols` + parameter. Note that when both the :py:attr:`inputCol` and :py:attr:`inputCols` parameters + are set, a log warning will be printed and only :py:attr:`inputCol` will take effect, while + :py:attr:`inputCols` will be ignored. The :py:attr:`splits` parameter is only used for single + column usage, and :py:attr:`splitsArray` is for multiple columns. >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), ... (float("nan"), 1.0), (float("nan"), 0.0)] From 76de8e62a5287dc1e37243d7d83220d3009d1b48 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 19 Dec 2017 11:08:33 +0800 Subject: [PATCH 06/11] update toListListFloat --- python/pyspark/ml/param/__init__.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 3840d169bc44..5b6b70292f09 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -141,11 +141,7 @@ def toListListFloat(value): """ if TypeConverters._can_convert_to_list(value): value = TypeConverters.toList(value) - if all(map(lambda v: TypeConverters._can_convert_to_list(v), value)): - ll = [] - for v in value: - ll.append([float(i) for i in TypeConverters.toList(v)]) - return ll + return [TypeConverters.toListFloat(v) for v in value] raise TypeError("Could not convert %s to list of list of floats" % value) @staticmethod From e869e75ce7e76a67bcad593243e2f08ca8cf23c4 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 16 Jan 2018 17:46:00 +0800 Subject: [PATCH 07/11] update tests --- python/pyspark/ml/feature.py | 62 +++++++++++++++++------------------- python/pyspark/ml/tests.py | 8 +++++ 2 files changed, 37 insertions(+), 33 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index cb32a0373687..1fd146f49f6a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -329,20 +329,22 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), ... (float("nan"), 1.0), (float("nan"), 0.0)] - >>> df = spark.createDataFrame(values, ["values", "numbers"]) + >>> df = spark.createDataFrame(values, ["values1", "values2"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], - ... inputCol="values", outputCol="buckets") - >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df).collect() - >>> len(bucketed) - 6 - >>> bucketed[0].buckets - 0.0 - >>> bucketed[1].buckets - 0.0 - >>> bucketed[2].buckets - 1.0 - >>> bucketed[3].buckets - 2.0 + ... inputCol="values1", outputCol="buckets") + >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df) + >>> bucketed.show(truncate=False) + +-------+-------+-------+ + |values1|values2|buckets| + +-------+-------+-------+ + |0.1 |0.0 |0.0 | + |0.4 |1.0 |0.0 | + |1.2 |1.3 |1.0 | + |1.5 |NaN |2.0 | + |NaN |1.0 |3.0 | + |NaN |0.0 |3.0 | + +-------+-------+-------+ + ... >>> bucketizer.setParams(outputCol="b").transform(df).head().b 0.0 >>> bucketizerPath = temp_path + "/bucketizer" @@ -355,26 +357,20 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu 4 >>> bucketizer2 = Bucketizer(splitsArray= ... [[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]], - ... inputCols=["values", "numbers"], outputCols=["buckets1", "buckets2"]) - >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df).collect() - >>> len(bucketed2) - 6 - >>> bucketed2[0].buckets1 - 0.0 - >>> bucketed2[1].buckets1 - 0.0 - >>> bucketed2[2].buckets1 - 1.0 - >>> bucketed2[3].buckets1 - 2.0 - >>> bucketed2[0].buckets2 - 0.0 - >>> bucketed2[1].buckets2 - 1.0 - >>> bucketed2[2].buckets2 - 1.0 - >>> bucketed2[3].buckets2 - 2.0 + ... inputCols=["values1", "values2"], outputCols=["buckets1", "buckets2"]) + >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df) + >>> bucketed2.show(truncate=False) + +-------+-------+--------+--------+ + |values1|values2|buckets1|buckets2| + +-------+-------+--------+--------+ + |0.1 |0.0 |0.0 |0.0 | + |0.4 |1.0 |0.0 |1.0 | + |1.2 |1.3 |1.0 |1.0 | + |1.5 |NaN |2.0 |2.0 | + |NaN |1.0 |3.0 |1.0 | + |NaN |0.0 |3.0 |0.0 | + +-------+-------+--------+--------+ + ... .. versionadded:: 1.4.0 """ diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 1af2b91da900..7660eaed4920 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -238,6 +238,14 @@ def test_bool(self): self.assertRaises(TypeError, lambda: LogisticRegression(fitIntercept=1)) self.assertRaises(TypeError, lambda: LogisticRegression(fitIntercept="false")) + def test_list_list_float(self): + b = Bucketizer(splitsArray=[[-0.1, 0.5, 3], [-5, 1.5]]) + self.assertEqual(b.getSplitsArray(), [[-0.1, 0.5, 3.0], [-5.0, 1.5]]) + self.assertTrue(all([type(v) == list for v in b.getSplitsArray()])) + self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[0]])) + self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[1]])) + self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=["a", 1.0])) + class PipelineTests(PySparkTestCase): From 9f20f5c0958070f40d65f0558d4290aae1535d04 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 16 Jan 2018 17:47:38 +0800 Subject: [PATCH 08/11] update tests --- python/pyspark/ml/tests.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 7660eaed4920..b8bddbd06f16 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -245,6 +245,7 @@ def test_list_list_float(self): self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[0]])) self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[1]])) self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=["a", 1.0])) + self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=[[-5, 1.5], ["a", 1.0]])) class PipelineTests(PySparkTestCase): From 734db506efdc41372032077ca9a6cf085cfb8920 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 16 Jan 2018 19:00:28 +0800 Subject: [PATCH 09/11] update tests and since --- python/pyspark/ml/feature.py | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1fd146f49f6a..6d37301399d5 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -332,18 +332,18 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> df = spark.createDataFrame(values, ["values1", "values2"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], ... inputCol="values1", outputCol="buckets") - >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df) + >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df.select("values1")) >>> bucketed.show(truncate=False) - +-------+-------+-------+ - |values1|values2|buckets| - +-------+-------+-------+ - |0.1 |0.0 |0.0 | - |0.4 |1.0 |0.0 | - |1.2 |1.3 |1.0 | - |1.5 |NaN |2.0 | - |NaN |1.0 |3.0 | - |NaN |0.0 |3.0 | - +-------+-------+-------+ + +-------+-------+ + |values1|buckets| + +-------+-------+ + |0.1 |0.0 | + |0.4 |0.0 | + |1.2 |1.0 | + |1.5 |2.0 | + |NaN |3.0 | + |NaN |3.0 | + +-------+-------+ ... >>> bucketizer.setParams(outputCol="b").transform(df).head().b 0.0 @@ -444,14 +444,14 @@ def getSplits(self): """ return self.getOrDefault(self.splits) - @since("2.3.0") + @since("2.4.0") def setSplitsArray(self, value): """ Sets the value of :py:attr:`splitsArray`. """ return self._set(splitsArray=value) - @since("2.3.0") + @since("2.4.0") def getSplitsArray(self): """ Gets the array of split points or its default value. From 014fb08ac279002203267bed65ebce2c980f7912 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Mon, 22 Jan 2018 15:59:24 +0800 Subject: [PATCH 10/11] revert to 2.3.0 --- python/pyspark/ml/feature.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 6d37301399d5..1719434d4d66 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -444,14 +444,14 @@ def getSplits(self): """ return self.getOrDefault(self.splits) - @since("2.4.0") + @since("2.3.0") def setSplitsArray(self, value): """ Sets the value of :py:attr:`splitsArray`. """ return self._set(splitsArray=value) - @since("2.4.0") + @since("2.3.0") def getSplitsArray(self): """ Gets the array of split points or its default value. From ad5d81de7dd059eab3e34a7085798ae19e3305ff Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 23 Jan 2018 15:14:20 +0800 Subject: [PATCH 11/11] update doc --- python/pyspark/ml/feature.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1719434d4d66..0ef00837b07a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -323,8 +323,7 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu Maps a column of continuous features to a column of feature buckets. Since 2.3.0, :py:class:`Bucketizer` can map multiple columns at once by setting the :py:attr:`inputCols` parameter. Note that when both the :py:attr:`inputCol` and :py:attr:`inputCols` parameters - are set, a log warning will be printed and only :py:attr:`inputCol` will take effect, while - :py:attr:`inputCols` will be ignored. The :py:attr:`splits` parameter is only used for single + are set, an Exception will be thrown. The :py:attr:`splits` parameter is only used for single column usage, and :py:attr:`splitsArray` is for multiple columns. >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")),