Skip to content

Commit a8a3e9b

Browse files
author
Nick Pentreath
committed
Revert "[SPARK-22797][PYSPARK] Bucketizer support multi-column"
This reverts commit c22eaa9.
1 parent dd8e257 commit a8a3e9b

File tree

3 files changed

+25
-99
lines changed

3 files changed

+25
-99
lines changed

python/pyspark/ml/feature.py

Lines changed: 25 additions & 80 deletions
Original file line numberDiff line numberDiff line change
@@ -317,33 +317,26 @@ class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable)
317317

318318

319319
@inherit_doc
320-
class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols,
321-
HasHandleInvalid, JavaMLReadable, JavaMLWritable):
322-
"""
323-
Maps a column of continuous features to a column of feature buckets. Since 2.3.0,
324-
:py:class:`Bucketizer` can map multiple columns at once by setting the :py:attr:`inputCols`
325-
parameter. Note that when both the :py:attr:`inputCol` and :py:attr:`inputCols` parameters
326-
are set, an Exception will be thrown. The :py:attr:`splits` parameter is only used for single
327-
column usage, and :py:attr:`splitsArray` is for multiple columns.
328-
329-
>>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")),
330-
... (float("nan"), 1.0), (float("nan"), 0.0)]
331-
>>> df = spark.createDataFrame(values, ["values1", "values2"])
320+
class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid,
321+
JavaMLReadable, JavaMLWritable):
322+
"""
323+
Maps a column of continuous features to a column of feature buckets.
324+
325+
>>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)]
326+
>>> df = spark.createDataFrame(values, ["values"])
332327
>>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")],
333-
... inputCol="values1", outputCol="buckets")
334-
>>> bucketed = bucketizer.setHandleInvalid("keep").transform(df.select("values1"))
335-
>>> bucketed.show(truncate=False)
336-
+-------+-------+
337-
|values1|buckets|
338-
+-------+-------+
339-
|0.1 |0.0 |
340-
|0.4 |0.0 |
341-
|1.2 |1.0 |
342-
|1.5 |2.0 |
343-
|NaN |3.0 |
344-
|NaN |3.0 |
345-
+-------+-------+
346-
...
328+
... inputCol="values", outputCol="buckets")
329+
>>> bucketed = bucketizer.setHandleInvalid("keep").transform(df).collect()
330+
>>> len(bucketed)
331+
6
332+
>>> bucketed[0].buckets
333+
0.0
334+
>>> bucketed[1].buckets
335+
0.0
336+
>>> bucketed[2].buckets
337+
1.0
338+
>>> bucketed[3].buckets
339+
2.0
347340
>>> bucketizer.setParams(outputCol="b").transform(df).head().b
348341
0.0
349342
>>> bucketizerPath = temp_path + "/bucketizer"
@@ -354,22 +347,6 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu
354347
>>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect()
355348
>>> len(bucketed)
356349
4
357-
>>> bucketizer2 = Bucketizer(splitsArray=
358-
... [[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]],
359-
... inputCols=["values1", "values2"], outputCols=["buckets1", "buckets2"])
360-
>>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df)
361-
>>> bucketed2.show(truncate=False)
362-
+-------+-------+--------+--------+
363-
|values1|values2|buckets1|buckets2|
364-
+-------+-------+--------+--------+
365-
|0.1 |0.0 |0.0 |0.0 |
366-
|0.4 |1.0 |0.0 |1.0 |
367-
|1.2 |1.3 |1.0 |1.0 |
368-
|1.5 |NaN |2.0 |2.0 |
369-
|NaN |1.0 |3.0 |1.0 |
370-
|NaN |0.0 |3.0 |0.0 |
371-
+-------+-------+--------+--------+
372-
...
373350
374351
.. versionadded:: 1.4.0
375352
"""
@@ -386,30 +363,14 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu
386363

387364
handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. " +
388365
"Options are 'skip' (filter out rows with invalid values), " +
389-
"'error' (throw an error), or 'keep' (keep invalid values in a " +
390-
"special additional bucket). Note that in the multiple column " +
391-
"case, the invalid handling is applied to all columns. That said " +
392-
"for 'error' it will throw an error if any invalids are found in " +
393-
"any column, for 'skip' it will skip rows with any invalids in " +
394-
"any columns, etc.",
366+
"'error' (throw an error), or 'keep' (keep invalid values in a special " +
367+
"additional bucket).",
395368
typeConverter=TypeConverters.toString)
396369

397-
splitsArray = Param(Params._dummy(), "splitsArray", "The array of split points for mapping " +
398-
"continuous features into buckets for multiple columns. For each input " +
399-
"column, with n+1 splits, there are n buckets. A bucket defined by " +
400-
"splits x,y holds values in the range [x,y) except the last bucket, " +
401-
"which also includes y. The splits should be of length >= 3 and " +
402-
"strictly increasing. Values at -inf, inf must be explicitly provided " +
403-
"to cover all Double values; otherwise, values outside the splits " +
404-
"specified will be treated as errors.",
405-
typeConverter=TypeConverters.toListListFloat)
406-
407370
@keyword_only
408-
def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error",
409-
splitsArray=None, inputCols=None, outputCols=None):
371+
def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"):
410372
"""
411-
__init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \
412-
splitsArray=None, inputCols=None, outputCols=None)
373+
__init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error")
413374
"""
414375
super(Bucketizer, self).__init__()
415376
self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Bucketizer", self.uid)
@@ -419,11 +380,9 @@ def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="er
419380

420381
@keyword_only
421382
@since("1.4.0")
422-
def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error",
423-
splitsArray=None, inputCols=None, outputCols=None):
383+
def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"):
424384
"""
425-
setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \
426-
splitsArray=None, inputCols=None, outputCols=None)
385+
setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error")
427386
Sets params for this Bucketizer.
428387
"""
429388
kwargs = self._input_kwargs
@@ -443,20 +402,6 @@ def getSplits(self):
443402
"""
444403
return self.getOrDefault(self.splits)
445404

446-
@since("2.3.0")
447-
def setSplitsArray(self, value):
448-
"""
449-
Sets the value of :py:attr:`splitsArray`.
450-
"""
451-
return self._set(splitsArray=value)
452-
453-
@since("2.3.0")
454-
def getSplitsArray(self):
455-
"""
456-
Gets the array of split points or its default value.
457-
"""
458-
return self.getOrDefault(self.splitsArray)
459-
460405

461406
@inherit_doc
462407
class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable):

python/pyspark/ml/param/__init__.py

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -134,16 +134,6 @@ def toListFloat(value):
134134
return [float(v) for v in value]
135135
raise TypeError("Could not convert %s to list of floats" % value)
136136

137-
@staticmethod
138-
def toListListFloat(value):
139-
"""
140-
Convert a value to list of list of floats, if possible.
141-
"""
142-
if TypeConverters._can_convert_to_list(value):
143-
value = TypeConverters.toList(value)
144-
return [TypeConverters.toListFloat(v) for v in value]
145-
raise TypeError("Could not convert %s to list of list of floats" % value)
146-
147137
@staticmethod
148138
def toListInt(value):
149139
"""

python/pyspark/ml/tests.py

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -238,15 +238,6 @@ def test_bool(self):
238238
self.assertRaises(TypeError, lambda: LogisticRegression(fitIntercept=1))
239239
self.assertRaises(TypeError, lambda: LogisticRegression(fitIntercept="false"))
240240

241-
def test_list_list_float(self):
242-
b = Bucketizer(splitsArray=[[-0.1, 0.5, 3], [-5, 1.5]])
243-
self.assertEqual(b.getSplitsArray(), [[-0.1, 0.5, 3.0], [-5.0, 1.5]])
244-
self.assertTrue(all([type(v) == list for v in b.getSplitsArray()]))
245-
self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[0]]))
246-
self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[1]]))
247-
self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=["a", 1.0]))
248-
self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=[[-5, 1.5], ["a", 1.0]]))
249-
250241

251242
class PipelineTests(PySparkTestCase):
252243

0 commit comments

Comments
 (0)