Skip to content
Next Next commit
type conversion for params
  • Loading branch information
sethah committed Mar 22, 2016
commit 8ab024208684b2d9df86ae1fdbfbcb5bca198131
12 changes: 8 additions & 4 deletions python/pyspark/ml/classification.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
from pyspark.ml.util import *
from pyspark.ml.wrapper import JavaEstimator, JavaModel
from pyspark.ml.param.shared import *
from pyspark.ml.param import TypeConverters
from pyspark.ml.regression import (
RandomForestParams, TreeEnsembleParams, DecisionTreeModel, TreeEnsembleModels)
from pyspark.mllib.common import inherit_doc
Expand Down Expand Up @@ -87,7 +88,8 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti

threshold = Param(Params._dummy(), "threshold",
"Threshold in binary classification prediction, in range [0, 1]." +
" If threshold and thresholds are both set, they must match.")
" If threshold and thresholds are both set, they must match.",
typeConverter=TypeConverters.convertToFloat)

@keyword_only
def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction",
Expand Down Expand Up @@ -652,7 +654,7 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H
"""

smoothing = Param(Params._dummy(), "smoothing", "The smoothing parameter, should be >= 0, " +
"default is 1.0")
"default is 1.0", typeConverter=TypeConverters.convertToFloat)
modelType = Param(Params._dummy(), "modelType", "The model type which is a string " +
"(case-sensitive). Supported options: multinomial (default) and bernoulli.")

Expand Down Expand Up @@ -782,11 +784,13 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol,

layers = Param(Params._dummy(), "layers", "Sizes of layers from input layer to output layer " +
"E.g., Array(780, 100, 10) means 780 inputs, one hidden layer with 100 " +
"neurons and output layer of 10 neurons, default is [1, 1].")
"neurons and output layer of 10 neurons, default is [1, 1].",
typeConverter=TypeConverters.convertToListInt)
blockSize = Param(Params._dummy(), "blockSize", "Block size for stacking input data in " +
"matrices. Data is stacked within partitions. If block size is more than " +
"remaining data in a partition then it is adjusted to the size of this " +
"data. Recommended size is between 10 and 1000, default is 128.")
"data. Recommended size is between 10 and 1000, default is 128.",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction",
Expand Down
12 changes: 8 additions & 4 deletions python/pyspark/ml/clustering.py
Original file line number Diff line number Diff line change
Expand Up @@ -87,12 +87,14 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol
.. versionadded:: 1.5.0
"""

k = Param(Params._dummy(), "k", "number of clusters to create")
k = Param(Params._dummy(), "k", "number of clusters to create",
typeConverter=TypeConverters.convertToInt)
initMode = Param(Params._dummy(), "initMode",
"the initialization algorithm. This can be either \"random\" to " +
"choose random points as initial cluster centers, or \"k-means||\" " +
"to use a parallel variant of k-means++")
initSteps = Param(Params._dummy(), "initSteps", "steps for k-means initialization mode")
initSteps = Param(Params._dummy(), "initSteps", "steps for k-means initialization mode",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, featuresCol="features", predictionCol="prediction", k=2,
Expand Down Expand Up @@ -227,10 +229,12 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte
.. versionadded:: 2.0.0
"""

k = Param(Params._dummy(), "k", "number of clusters to create")
k = Param(Params._dummy(), "k", "number of clusters to create",
typeConverter=TypeConverters.convertToInt)
minDivisibleClusterSize = Param(Params._dummy(), "minDivisibleClusterSize",
"the minimum number of points (if >= 1.0) " +
"or the minimum proportion")
"or the minimum proportion",
typeConverter=TypeConverters.convertToFloat)

@keyword_only
def __init__(self, featuresCol="features", predictionCol="prediction", maxIter=20,
Expand Down
67 changes: 43 additions & 24 deletions python/pyspark/ml/feature.py
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,8 @@ class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java
"""

threshold = Param(Params._dummy(), "threshold",
"threshold in binary classification prediction, in range [0, 1]")
"threshold in binary classification prediction, in range [0, 1]",
typeConverter=TypeConverters.convertToFloat)

@keyword_only
def __init__(self, threshold=0.0, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -159,7 +160,8 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav
"range [x,y) except the last bucket, which also includes y. The splits " +
"should be 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.")
"specified will be treated as errors.",
typeConverter=TypeConverters.convertToListFloat)

@keyword_only
def __init__(self, splits=None, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -243,15 +245,17 @@ class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable,
" threshold are ignored. If this is an integer >= 1, then this specifies a count (of" +
" times the term must appear in the document); if this is a double in [0,1), then this " +
"specifies a fraction (out of the document's token count). Note that the parameter is " +
"only used in transform of CountVectorizerModel and does not affect fitting. Default 1.0")
"only used in transform of CountVectorizerModel and does not affect fitting. Default 1.0",
typeConverter=TypeConverters.convertToInt)
minDF = Param(
Params._dummy(), "minDF", "Specifies the minimum number of" +
" different documents a term must appear in to be included in the vocabulary." +
" If this is an integer >= 1, this specifies the number of documents the term must" +
" appear in; if this is a double in [0,1), then this specifies the fraction of documents." +
" Default 1.0")
" Default 1.0", typeConverter=TypeConverters.convertToInt)
vocabSize = Param(
Params._dummy(), "vocabSize", "max size of the vocabulary. Default 1 << 18.")
Params._dummy(), "vocabSize", "max size of the vocabulary. Default 1 << 18.",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, minTF=1.0, minDF=1.0, vocabSize=1 << 18, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -442,7 +446,8 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada
"""

scalingVec = Param(Params._dummy(), "scalingVec", "vector for hadamard product, " +
"it must be MLlib Vector type.")
"it must be MLlib Vector type.",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can remove this restriction in the doc now.

typeConverter=TypeConverters.convertToVector)

@keyword_only
def __init__(self, scalingVec=None, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -564,7 +569,8 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab
"""

minDocFreq = Param(Params._dummy(), "minDocFreq",
"minimum of documents in which a term should appear for filtering")
"minimum of documents in which a term should appear for filtering",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, minDocFreq=0, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -746,8 +752,10 @@ class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav
.. versionadded:: 1.6.0
"""

min = Param(Params._dummy(), "min", "Lower bound of the output feature range")
max = Param(Params._dummy(), "max", "Upper bound of the output feature range")
min = Param(Params._dummy(), "min", "Lower bound of the output feature range",
typeConverter=TypeConverters.convertToInt)
max = Param(Params._dummy(), "max", "Upper bound of the output feature range",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, min=0.0, max=1.0, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -870,7 +878,8 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr
.. versionadded:: 1.5.0
"""

n = Param(Params._dummy(), "n", "number of elements per n-gram (>=1)")
n = Param(Params._dummy(), "n", "number of elements per n-gram (>=1)",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, n=2, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -936,7 +945,8 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav
.. versionadded:: 1.4.0
"""

p = Param(Params._dummy(), "p", "the p norm value.")
p = Param(Params._dummy(), "p", "the p norm value.",
typeConverter=TypeConverters.convertToFloat)

@keyword_only
def __init__(self, p=2.0, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -1085,7 +1095,8 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead
.. versionadded:: 1.4.0
"""

degree = Param(Params._dummy(), "degree", "the polynomial degree to expand (>= 1)")
degree = Param(Params._dummy(), "degree", "the polynomial degree to expand (>= 1)",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, degree=2, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -1163,7 +1174,8 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav
# a placeholder to make it appear in the generated doc
numBuckets = Param(Params._dummy(), "numBuckets",
"Maximum number of buckets (quantiles, or " +
"categories) into which data points are grouped. Must be >= 2. Default 2.")
"categories) into which data points are grouped. Must be >= 2. Default 2.",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None):
Expand Down Expand Up @@ -1255,7 +1267,8 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable,
.. versionadded:: 1.4.0
"""

minTokenLength = Param(Params._dummy(), "minTokenLength", "minimum token length (>= 0)")
minTokenLength = Param(Params._dummy(), "minTokenLength", "minimum token length (>= 0)",
typeConverter=TypeConverters.convertToInt)
gaps = Param(Params._dummy(), "gaps", "whether regex splits on gaps (True) or matches tokens")
pattern = Param(Params._dummy(), "pattern", "regex pattern (Java dialect) used for tokenizing")
toLowercase = Param(Params._dummy(), "toLowercase", "whether to convert all characters to " +
Expand Down Expand Up @@ -1628,7 +1641,8 @@ class IndexToString(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable,

labels = Param(Params._dummy(), "labels",
"Optional array of labels specifying index-string mapping." +
" If not provided or if empty, then metadata from inputCol is used instead.")
" If not provided or if empty, then metadata from inputCol is used instead.",
typeConverter=TypeConverters.convertToList)

@keyword_only
def __init__(self, inputCol=None, outputCol=None, labels=None):
Expand Down Expand Up @@ -1689,7 +1703,8 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadabl
.. versionadded:: 1.6.0
"""

stopWords = Param(Params._dummy(), "stopWords", "The words to be filtered out")
stopWords = Param(Params._dummy(), "stopWords", "The words to be filtered out",
typeConverter=TypeConverters.convertToList)
caseSensitive = Param(Params._dummy(), "caseSensitive", "whether to do a case sensitive " +
"comparison over the stop words")

Expand Down Expand Up @@ -1930,7 +1945,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja
maxCategories = Param(Params._dummy(), "maxCategories",
"Threshold for the number of values a categorical feature can take " +
"(>= 2). If a feature is found to have > maxCategories values, then " +
"it is declared continuous.")
"it is declared continuous.", typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, maxCategories=20, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -2035,11 +2050,12 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J
"""

indices = Param(Params._dummy(), "indices", "An array of indices to select features from " +
"a vector column. There can be no overlap with names.")
"a vector column. There can be no overlap with names.",
typeConverter=TypeConverters.convertToListInt)
names = Param(Params._dummy(), "names", "An array of feature names to select features from " +
"a vector column. These names must be specified by ML " +
"org.apache.spark.ml.attribute.Attribute. There can be no overlap with " +
"indices.")
"indices.", typeConverter=TypeConverters.convertToList)

@keyword_only
def __init__(self, inputCol=None, outputCol=None, indices=None, names=None):
Expand Down Expand Up @@ -2147,12 +2163,14 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, Has
"""

vectorSize = Param(Params._dummy(), "vectorSize",
"the dimension of codes after transforming from words")
"the dimension of codes after transforming from words",
typeConverter=TypeConverters.convertToInt)
numPartitions = Param(Params._dummy(), "numPartitions",
"number of partitions for sentences of words")
"number of partitions for sentences of words",
typeConverter=TypeConverters.convertToInt)
minCount = Param(Params._dummy(), "minCount",
"the minimum number of times a token must appear to be included in the " +
"word2vec model's vocabulary")
"word2vec model's vocabulary", typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1,
Expand Down Expand Up @@ -2293,7 +2311,8 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab
.. versionadded:: 1.5.0
"""

k = Param(Params._dummy(), "k", "the number of principal components")
k = Param(Params._dummy(), "k", "the number of principal components",
typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, k=None, inputCol=None, outputCol=None):
Expand Down Expand Up @@ -2516,7 +2535,7 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja
Param(Params._dummy(), "numTopFeatures",
"Number of features that selector will select, ordered by statistics value " +
"descending. If the number of features is < numTopFeatures, then this will select " +
"all features.")
"all features.", typeConverter=TypeConverters.convertToInt)

@keyword_only
def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, labelCol="label"):
Expand Down
Loading