[SPARK-14615][ML][FOLLOWUP] Fix Python examples to use the new ML Vector and Matrix APIs in the ML pipeline based algorithms
## What changes were proposed in this pull request? This PR fixes Python examples to use the new ML Vector and Matrix APIs in the ML pipeline based algorithms. I firstly executed this shell command, `grep -r "from pyspark.mllib" .` and then executed them all. Some of tests in `ml` produced the error messages as below: ``` pyspark.sql.utils.IllegalArgumentException: u'requirement failed: Input type must be VectorUDT but got org.apache.spark.mllib.linalg.VectorUDTf71b0bce.' ``` So, I fixed them to use new ones just identically with some Python tests fixed in https://github.com/apache/spark/pull/12627 ## How was this patch tested? Manually tested for all the examples listed by `grep -r "from pyspark.mllib" .`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #13393 from HyukjinKwon/SPARK-14615.
This commit is contained in:
parent
bba5d7999f
commit
99f3c82776
|
@ -19,7 +19,7 @@ from __future__ import print_function
|
|||
|
||||
# $example on$
|
||||
from pyspark.ml.regression import AFTSurvivalRegression
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ from __future__ import print_function
|
|||
from pyspark.sql import SparkSession
|
||||
# $example on$
|
||||
from pyspark.ml.feature import ChiSqSelector
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
# $example off$
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
|
|
@ -19,7 +19,7 @@ from __future__ import print_function
|
|||
|
||||
# $example on$
|
||||
from pyspark.ml.feature import DCT
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ from __future__ import print_function
|
|||
|
||||
# $example on$
|
||||
from pyspark.ml.feature import ElementwiseProduct
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ Estimator Transformer Param Example.
|
|||
"""
|
||||
|
||||
# $example on$
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
from pyspark.ml.classification import LogisticRegression
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
|
|
@ -19,7 +19,7 @@ from __future__ import print_function
|
|||
|
||||
# $example on$
|
||||
from pyspark.ml.feature import PCA
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ from __future__ import print_function
|
|||
|
||||
# $example on$
|
||||
from pyspark.ml.feature import PolynomialExpansion
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
||||
|
|
|
@ -21,9 +21,8 @@ import pprint
|
|||
import sys
|
||||
|
||||
from pyspark.ml.classification import LogisticRegression
|
||||
from pyspark.mllib.linalg import DenseVector
|
||||
from pyspark.mllib.regression import LabeledPoint
|
||||
from pyspark.sql import SparkSession
|
||||
from pyspark.ml.linalg import DenseVector
|
||||
from pyspark.sql import Row, SparkSession
|
||||
|
||||
"""
|
||||
A simple example demonstrating ways to specify parameters for Estimators and Transformers.
|
||||
|
@ -42,10 +41,10 @@ if __name__ == "__main__":
|
|||
# A LabeledPoint is an Object with two fields named label and features
|
||||
# and Spark SQL identifies these fields and creates the schema appropriately.
|
||||
training = spark.createDataFrame([
|
||||
LabeledPoint(1.0, DenseVector([0.0, 1.1, 0.1])),
|
||||
LabeledPoint(0.0, DenseVector([2.0, 1.0, -1.0])),
|
||||
LabeledPoint(0.0, DenseVector([2.0, 1.3, 1.0])),
|
||||
LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))])
|
||||
Row(label=1.0, features=DenseVector([0.0, 1.1, 0.1])),
|
||||
Row(label=0.0, features=DenseVector([2.0, 1.0, -1.0])),
|
||||
Row(label=0.0, features=DenseVector([2.0, 1.3, 1.0])),
|
||||
Row(label=1.0, features=DenseVector([0.0, 1.2, -0.5]))])
|
||||
|
||||
# Create a LogisticRegression instance with maxIter = 10.
|
||||
# This instance is an Estimator.
|
||||
|
@ -77,9 +76,9 @@ if __name__ == "__main__":
|
|||
|
||||
# prepare test data.
|
||||
test = spark.createDataFrame([
|
||||
LabeledPoint(1.0, DenseVector([-1.0, 1.5, 1.3])),
|
||||
LabeledPoint(0.0, DenseVector([3.0, 2.0, -0.1])),
|
||||
LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))])
|
||||
Row(label=1.0, features=DenseVector([-1.0, 1.5, 1.3])),
|
||||
Row(label=0.0, features=DenseVector([3.0, 2.0, -0.1])),
|
||||
Row(label=0.0, features=DenseVector([0.0, 2.2, -1.5]))])
|
||||
|
||||
# Make predictions on test data using the Transformer.transform() method.
|
||||
# LogisticRegressionModel.transform will only use the 'features' column.
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
from __future__ import print_function
|
||||
|
||||
# $example on$
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
from pyspark.ml.feature import VectorAssembler
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
|
|
@ -19,7 +19,7 @@ from __future__ import print_function
|
|||
|
||||
# $example on$
|
||||
from pyspark.ml.feature import VectorSlicer
|
||||
from pyspark.mllib.linalg import Vectors
|
||||
from pyspark.ml.linalg import Vectors
|
||||
from pyspark.sql.types import Row
|
||||
# $example off$
|
||||
from pyspark.sql import SparkSession
|
||||
|
|
Loading…
Reference in a new issue