aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--python/pyspark/ml/feature.py148
1 files changed, 142 insertions, 6 deletions
diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py
index 92db8df802..f41d72f877 100644
--- a/python/pyspark/ml/feature.py
+++ b/python/pyspark/ml/feature.py
@@ -26,12 +26,13 @@ from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer, _jvm
from pyspark.mllib.common import inherit_doc
from pyspark.mllib.linalg import _convert_to_vector
-__all__ = ['Binarizer', 'Bucketizer', 'DCT', 'ElementwiseProduct', 'HashingTF', 'IDF', 'IDFModel',
- 'IndexToString', 'MinMaxScaler', 'MinMaxScalerModel', 'NGram', 'Normalizer',
- 'OneHotEncoder', 'PCA', 'PCAModel', 'PolynomialExpansion', 'RegexTokenizer',
- 'RFormula', 'RFormulaModel', 'SQLTransformer', 'StandardScaler', 'StandardScalerModel',
- 'StopWordsRemover', 'StringIndexer', 'StringIndexerModel', 'Tokenizer',
- 'VectorAssembler', 'VectorIndexer', 'VectorSlicer', 'Word2Vec', 'Word2VecModel']
+__all__ = ['Binarizer', 'Bucketizer', 'CountVectorizer', 'CountVectorizerModel', 'DCT',
+ 'ElementwiseProduct', 'HashingTF', 'IDF', 'IDFModel', 'IndexToString', 'MinMaxScaler',
+ 'MinMaxScalerModel', 'NGram', 'Normalizer', 'OneHotEncoder', 'PCA', 'PCAModel',
+ 'PolynomialExpansion', 'RegexTokenizer', 'RFormula', 'RFormulaModel', 'SQLTransformer',
+ 'StandardScaler', 'StandardScalerModel', 'StopWordsRemover', 'StringIndexer',
+ 'StringIndexerModel', 'Tokenizer', 'VectorAssembler', 'VectorIndexer', 'VectorSlicer',
+ 'Word2Vec', 'Word2VecModel']
@inherit_doc
@@ -172,6 +173,141 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol):
@inherit_doc
+class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol):
+ """
+ .. note:: Experimental
+
+ Extracts a vocabulary from document collections and generates a :py:attr:`CountVectorizerModel`.
+ >>> df = sqlContext.createDataFrame(
+ ... [(0, ["a", "b", "c"]), (1, ["a", "b", "b", "c", "a"])],
+ ... ["label", "raw"])
+ >>> cv = CountVectorizer(inputCol="raw", outputCol="vectors")
+ >>> model = cv.fit(df)
+ >>> model.transform(df).show(truncate=False)
+ +-----+---------------+-------------------------+
+ |label|raw |vectors |
+ +-----+---------------+-------------------------+
+ |0 |[a, b, c] |(3,[0,1,2],[1.0,1.0,1.0])|
+ |1 |[a, b, b, c, a]|(3,[0,1,2],[2.0,2.0,1.0])|
+ +-----+---------------+-------------------------+
+ ...
+ >>> sorted(map(str, model.vocabulary))
+ ['a', 'b', 'c']
+ """
+
+ # a placeholder to make it appear in the generated doc
+ minTF = Param(
+ Params._dummy(), "minTF", "Filter to ignore rare words in" +
+ " a document. For each document, terms with frequency/count less than the given" +
+ " 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")
+ 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")
+ vocabSize = Param(
+ Params._dummy(), "vocabSize", "max size of the vocabulary. Default 1 << 18.")
+
+ @keyword_only
+ def __init__(self, minTF=1.0, minDF=1.0, vocabSize=1 << 18, inputCol=None, outputCol=None):
+ """
+ __init__(self, minTF=1.0, minDF=1.0, vocabSize=1 << 18, inputCol=None, outputCol=None)
+ """
+ super(CountVectorizer, self).__init__()
+ self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.CountVectorizer",
+ self.uid)
+ self.minTF = Param(
+ self, "minTF", "Filter to ignore rare words in" +
+ " a document. For each document, terms with frequency/count less than the given" +
+ " 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")
+ self.minDF = Param(
+ self, "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")
+ self.vocabSize = Param(
+ self, "vocabSize", "max size of the vocabulary. Default 1 << 18.")
+ self._setDefault(minTF=1.0, minDF=1.0, vocabSize=1 << 18)
+ kwargs = self.__init__._input_kwargs
+ self.setParams(**kwargs)
+
+ @keyword_only
+ def setParams(self, minTF=1.0, minDF=1.0, vocabSize=1 << 18, inputCol=None, outputCol=None):
+ """
+ setParams(self, minTF=1.0, minDF=1.0, vocabSize=1 << 18, inputCol=None, outputCol=None)
+ Set the params for the CountVectorizer
+ """
+ kwargs = self.setParams._input_kwargs
+ return self._set(**kwargs)
+
+ def setMinTF(self, value):
+ """
+ Sets the value of :py:attr:`minTF`.
+ """
+ self._paramMap[self.minTF] = value
+ return self
+
+ def getMinTF(self):
+ """
+ Gets the value of minTF or its default value.
+ """
+ return self.getOrDefault(self.minTF)
+
+ def setMinDF(self, value):
+ """
+ Sets the value of :py:attr:`minDF`.
+ """
+ self._paramMap[self.minDF] = value
+ return self
+
+ def getMinDF(self):
+ """
+ Gets the value of minDF or its default value.
+ """
+ return self.getOrDefault(self.minDF)
+
+ def setVocabSize(self, value):
+ """
+ Sets the value of :py:attr:`vocabSize`.
+ """
+ self._paramMap[self.vocabSize] = value
+ return self
+
+ def getVocabSize(self):
+ """
+ Gets the value of vocabSize or its default value.
+ """
+ return self.getOrDefault(self.vocabSize)
+
+ def _create_model(self, java_model):
+ return CountVectorizerModel(java_model)
+
+
+class CountVectorizerModel(JavaModel):
+ """
+ .. note:: Experimental
+
+ Model fitted by CountVectorizer.
+ """
+
+ @property
+ def vocabulary(self):
+ """
+ An array of terms in the vocabulary.
+ """
+ return self._call_java("vocabulary")
+
+
+@inherit_doc
class DCT(JavaTransformer, HasInputCol, HasOutputCol):
"""
.. note:: Experimental