org.apache.spark.ml

feature

package feature

Feature transformers

The ml.feature package provides common feature transformers that help convert raw data or features into more suitable forms for model fitting. Most feature transformers are implemented as Transformers, which transform one DataFrame into another, e.g., HashingTF. Some feature transformers are implemented as Estimators, because the transformation requires some aggregated information of the dataset, e.g., document frequencies in IDF. For those feature transformers, calling Estimator!.fit is required to obtain the model first, e.g., IDFModel, in order to apply transformation. The transformation is usually done by appending new columns to the input DataFrame, so all input columns are carried over.

We try to make each transformer minimal, so it becomes flexible to assemble feature transformation pipelines. Pipeline can be used to chain feature transformers, and VectorAssembler can be used to combine multiple feature transformations, for example:

import org.apache.spark.ml.feature._
import org.apache.spark.ml.Pipeline

// a DataFrame with three columns: id (integer), text (string), and rating (double).
val df = sqlContext.createDataFrame(Seq(
  (0, "Hi I heard about Spark", 3.0),
  (1, "I wish Java could use case classes", 4.0),
  (2, "Logistic regression models are neat", 4.0)
)).toDF("id", "text", "rating")

// define feature transformers
val tok = new RegexTokenizer()
  .setInputCol("text")
  .setOutputCol("words")
val sw = new StopWordsRemover()
  .setInputCol("words")
  .setOutputCol("filtered_words")
val tf = new HashingTF()
  .setInputCol("filtered_words")
  .setOutputCol("tf")
  .setNumFeatures(10000)
val idf = new IDF()
  .setInputCol("tf")
  .setOutputCol("tf_idf")
val assembler = new VectorAssembler()
  .setInputCols(Array("tf_idf", "rating"))
  .setOutputCol("features")

// assemble and fit the feature transformation pipeline
val pipeline = new Pipeline()
  .setStages(Array(tok, sw, tf, idf, assembler))
val model = pipeline.fit(df)

// save transformed features with raw data
model.transform(df)
  .select("id", "text", "rating", "features")
  .write.format("parquet").save("/output/path")

Some feature transformers implemented in MLlib are inspired by those implemented in scikit-learn. The major difference is that most scikit-learn feature transformers operate eagerly on the entire input dataset, while MLlib's feature transformers operate lazily on individual columns, which is more efficient and flexible to handle large and complex datasets.

See also

scikit-learn.preprocessing

Linear Supertypes
AnyRef, Any
Ordering
  1. Alphabetic
  2. By inheritance
Inherited
  1. feature
  2. AnyRef
  3. Any
  1. Hide All
  2. Show all
Learn more about member selection
Visibility
  1. Public
  2. All

Type Members

  1. final class Binarizer extends Transformer with HasInputCol with HasOutputCol

    :: Experimental :: Binarize a column of continuous features given a threshold.

  2. final class Bucketizer extends Model[Bucketizer] with HasInputCol with HasOutputCol

    :: Experimental :: Bucketizer maps a column of continuous features to a column of feature buckets.

  3. class CountVectorizer extends Estimator[CountVectorizerModel] with CountVectorizerParams

    :: Experimental :: Extracts a vocabulary from document collections and generates a CountVectorizerModel.

  4. class CountVectorizerModel extends Model[CountVectorizerModel] with CountVectorizerParams

    :: Experimental :: Converts a text document to a sparse vector of token counts.

  5. class DCT extends UnaryTransformer[Vector, Vector, DCT]

    :: Experimental :: A feature transformer that takes the 1D discrete cosine transform of a real vector.

  6. class ElementwiseProduct extends UnaryTransformer[Vector, Vector, ElementwiseProduct]

    :: Experimental :: Outputs the Hadamard product (i.

  7. class HashingTF extends Transformer with HasInputCol with HasOutputCol

    :: Experimental :: Maps a sequence of terms to their term frequencies using the hashing trick.

  8. final class IDF extends Estimator[IDFModel] with IDFBase

    :: Experimental :: Compute the Inverse Document Frequency (IDF) given a collection of documents.

  9. class IDFModel extends Model[IDFModel] with IDFBase

    :: Experimental :: Model fitted by IDF.

  10. class IndexToString extends Transformer with HasInputCol with HasOutputCol

    :: Experimental :: A Transformer that maps a column of string indices back to a new column of corresponding string values using either the ML attributes of the input column, or if provided using the labels supplied by the user.

  11. class MinMaxScaler extends Estimator[MinMaxScalerModel] with MinMaxScalerParams

    :: Experimental :: Rescale each feature individually to a common range [min, max] linearly using column summary statistics, which is also known as min-max normalization or Rescaling.

  12. class MinMaxScalerModel extends Model[MinMaxScalerModel] with MinMaxScalerParams

    :: Experimental :: Model fitted by MinMaxScaler.

  13. class NGram extends UnaryTransformer[Seq[String], Seq[String], NGram]

    :: Experimental :: A feature transformer that converts the input array of strings into an array of n-grams.

  14. class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer]

    :: Experimental :: Normalize a vector to have unit norm using the given p-norm.

  15. class OneHotEncoder extends Transformer with HasInputCol with HasOutputCol

    :: Experimental :: A one-hot encoder that maps a column of category indices to a column of binary vectors, with at most a single one-value per row that indicates the input category index.

  16. class PCA extends Estimator[PCAModel] with PCAParams

    :: Experimental :: PCA trains a model to project vectors to a low-dimensional space using PCA.

  17. class PCAModel extends Model[PCAModel] with PCAParams

    :: Experimental :: Model fitted by PCA.

  18. class PolynomialExpansion extends UnaryTransformer[Vector, Vector, PolynomialExpansion]

    :: Experimental :: Perform feature expansion in a polynomial space.

  19. class RFormula extends Estimator[RFormulaModel] with RFormulaBase

    :: Experimental :: Implements the transforms required for fitting a dataset against an R model formula.

  20. class RFormulaModel extends Model[RFormulaModel] with RFormulaBase

    :: Experimental :: A fitted RFormula.

  21. class RegexTokenizer extends UnaryTransformer[String, Seq[String], RegexTokenizer]

    :: Experimental :: A regex based tokenizer that extracts tokens either by using the provided regex pattern to split the text (default) or repeatedly matching the regex (if gaps is false).

  22. class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerParams

    :: Experimental :: Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set.

  23. class StandardScalerModel extends Model[StandardScalerModel] with StandardScalerParams

    :: Experimental :: Model fitted by StandardScaler.

  24. class StopWordsRemover extends Transformer with HasInputCol with HasOutputCol

    :: Experimental :: A feature transformer that filters out stop words from input.

  25. class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase

    :: Experimental :: A label indexer that maps a string column of labels to an ML column of label indices.

  26. class StringIndexerModel extends Model[StringIndexerModel] with StringIndexerBase

    :: Experimental :: Model fitted by StringIndexer.

  27. class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer]

    :: Experimental :: A tokenizer that converts the input string to lowercase and then splits it by white spaces.

  28. class VectorAssembler extends Transformer with HasInputCols with HasOutputCol

    :: Experimental :: A feature transformer that merges multiple columns into a vector column.

  29. class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerParams

    :: Experimental :: Class for indexing categorical feature columns in a dataset of Vector.

  30. class VectorIndexerModel extends Model[VectorIndexerModel] with VectorIndexerParams

    :: Experimental :: Transform categorical features to use 0-based indices instead of their original values.

  31. final class VectorSlicer extends Transformer with HasInputCol with HasOutputCol

    :: Experimental :: This class takes a feature vector and outputs a new feature vector with a subarray of the original features.

  32. final class Word2Vec extends Estimator[Word2VecModel] with Word2VecBase

    :: Experimental :: Word2Vec trains a model of Map(String, Vector), i.

  33. class Word2VecModel extends Model[Word2VecModel] with Word2VecBase

    :: Experimental :: Model fitted by Word2Vec.

Inherited from AnyRef

Inherited from Any

Members