Skip to content

Commit

Permalink
re-org:
Browse files Browse the repository at this point in the history
1. move SimpleTokenizer to examples
2. move LR to classification, HashingTF and StandardScaler to feature, CV and ParamGridBuilder to tuning
3. define SimpleTransformer
  • Loading branch information
mengxr committed Nov 9, 2014
1 parent 7772430 commit e246f29
Show file tree
Hide file tree
Showing 16 changed files with 362 additions and 222 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.examples.ml;

import java.util.List;

import com.google.common.collect.Lists;

import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.ml.Pipeline;
import org.apache.spark.ml.PipelineModel;
import org.apache.spark.ml.PipelineStage;
import org.apache.spark.ml.classification.LogisticRegression;
import org.apache.spark.ml.feature.HashingTF;
import org.apache.spark.sql.api.java.JavaSQLContext;
import org.apache.spark.sql.api.java.JavaSchemaRDD;
import org.apache.spark.sql.api.java.Row;
import org.apache.spark.SparkConf;

/**
* A simple text classification pipeline that recognizes "spark" from input text. It uses the Java
* bean classes [[LabeledDocument]] and [[Document]], and the tokenizer [[SimpleTokenizer]] defined
* in the Scalar counterpart of this example [[SimpleTextClassificationPipeline]]. Run with
* <pre>
* bin/run-example ml.JavaSimpleTextClassificationPipeline
* </pre>
*/
public class JavaSimpleTextClassificationPipeline {

public static void main(String[] args) {
SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline");
JavaSparkContext jsc = new JavaSparkContext(conf);
JavaSQLContext jsql = new JavaSQLContext(jsc);

// Prepare training documents, which are labeled.
List<LabeledDocument> localTraining = Lists.newArrayList(
new LabeledDocument(0L, "a b c d e spark", 1.0),
new LabeledDocument(1L, "b d", 0.0),
new LabeledDocument(2L, "spark f g h", 1.0),
new LabeledDocument(3L, "hadoop mapreduce", 0.0));
JavaSchemaRDD training =
jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class);

// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
SimpleTokenizer tokenizer = new SimpleTokenizer()
.setInputCol("text")
.setOutputCol("words");
HashingTF hashingTF = new HashingTF()
.setNumFeatures(1000)
.setInputCol(tokenizer.getOutputCol())
.setOutputCol("features");
LogisticRegression lr = new LogisticRegression()
.setMaxIter(10)
.setRegParam(0.01);
Pipeline pipeline = new Pipeline()
.setStages(new PipelineStage[] {tokenizer, hashingTF, lr});

// Fit the pipeline to training documents.
PipelineModel model = pipeline.fit(training);

// Prepare test documents, which are unlabeled.
List<Document> localTest = Lists.newArrayList(
new Document(4L, "spark i j k"),
new Document(5L, "l m n"),
new Document(6L, "mapreduce spark"),
new Document(7L, "apache hadoop"));
JavaSchemaRDD test =
jsql.applySchema(jsc.parallelize(localTest), Document.class);

// Make predictions on test documents.
model.transform(test).registerAsTable("prediction");
JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction");
for (Row r: predictions.collect()) {
System.out.println(r);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.examples.ml

import scala.beans.BeanInfo

import org.apache.spark.ml.classification.LogisticRegression
import org.apache.spark.ml.feature.HashingTF
import org.apache.spark.ml.{Pipeline, SimpleTransformer}
import org.apache.spark.sql.SQLContext
import org.apache.spark.{SparkConf, SparkContext}

@BeanInfo
case class LabeledDocument(id: Long, text: String, label: Double)

@BeanInfo
case class Document(id: Long, text: String)

/**
* A tokenizer that converts the input string to lowercase and then splits it by white spaces.
*/
class SimpleTokenizer extends SimpleTransformer[String, Seq[String], SimpleTokenizer]
with Serializable {
override def createTransformFunc: String => Seq[String] = _.toLowerCase.split("\\s")
}

/**
* A simple text classification pipeline that recognizes "spark" from input text. This is to show
* how to define a simple tokenizer and then use it as part of a ML pipeline. Run with
* {{{
* bin/run-example ml.SimpleTextClassificationPipeline
* }}}
*/
object SimpleTextClassificationPipeline {

def main(args: Array[String]) {
val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline")
val sc = new SparkContext(conf)
val sqlContext = new SQLContext(sc)
import sqlContext._

// Prepare training documents, which are labeled.
val training = sparkContext.parallelize(Seq(
LabeledDocument(0L, "a b c d e spark", 1.0),
LabeledDocument(1L, "b d", 0.0),
LabeledDocument(2L, "spark f g h", 1.0),
LabeledDocument(3L, "hadoop mapreduce", 0.0)))

// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr.
val tokenizer = new SimpleTokenizer()
.setInputCol("text")
.setOutputCol("words")
val hashingTF = new HashingTF()
.setNumFeatures(1000)
.setInputCol(tokenizer.getOutputCol)
.setOutputCol("features")
val lr = new LogisticRegression()
.setMaxIter(10)
.setRegParam(0.01)
val pipeline = new Pipeline()
.setStages(Array(tokenizer, hashingTF, lr))

// Fit the pipeline to training documents.
val model = pipeline.fit(training)

// Prepare test documents, which are unlabeled.
val test = sparkContext.parallelize(Seq(
Document(4L, "spark i j k"),
Document(5L, "l m n"),
Document(6L, "mapreduce spark"),
Document(7L, "apache hadoop")))

// Make predictions on test documents.
model.transform(test)
.select('id, 'text, 'score, 'prediction)
.collect()
.foreach(println)
}
}
12 changes: 2 additions & 10 deletions mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,12 @@ package org.apache.spark.ml
import java.util.UUID

/**
* Something with a unique id.
* Object with a unique id.
*/
trait Identifiable extends Serializable {

/**
* A unique id for the object.
*/
val uid: String = this.getClass.getSimpleName + "-" + Identifiable.randomUid
}

object Identifiable {

/**
* Returns a random uid, drawn uniformly from 4+ billion candidates.
*/
private def randomUid: String = UUID.randomUUID().toString.take(8)
val uid: String = this.getClass.getSimpleName + "-" + UUID.randomUUID().toString.take(8)
}
2 changes: 1 addition & 1 deletion mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.SchemaRDD
/**
* A stage in a pipeline, either an Estimator or an Transformer.
*/
trait PipelineStage extends Identifiable
abstract class PipelineStage

/**
* A simple pipeline, which acts as an estimator.
Expand Down
25 changes: 24 additions & 1 deletion mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,13 @@
package org.apache.spark.ml

import scala.annotation.varargs
import scala.reflect.runtime.universe.TypeTag

import org.apache.spark.ml.param.{ParamMap, ParamPair, Params}
import org.apache.spark.ml.param._
import org.apache.spark.sql.SchemaRDD
import org.apache.spark.sql.api.java.JavaSchemaRDD
import org.apache.spark.sql.catalyst.analysis.Star
import org.apache.spark.sql.catalyst.dsl._

/**
* Abstract class for transformers that transform one dataset into another.
Expand Down Expand Up @@ -60,3 +63,23 @@ abstract class Transformer extends PipelineStage with Params {
transform(dataset.schemaRDD, paramMap).toJavaSchemaRDD
}
}

/**
* Abstract class for transformers that take one input column, apply transformation, and output the
* result as a new column.
*/
abstract class SimpleTransformer[IN, OUT: TypeTag, SELF <: SimpleTransformer[IN, OUT, SELF]]
extends Transformer with HasInputCol with HasOutputCol {

def setInputCol(value: String): SELF = { set(inputCol, value); this.asInstanceOf[SELF] }
def setOutputCol(value: String): SELF = { set(outputCol, value); this.asInstanceOf[SELF] }

def createTransformFunc: IN => OUT

override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = {
import dataset.sqlContext._
val map = this.paramMap ++ paramMap
val udf: IN => OUT = this.createTransformFunc
dataset.select(Star(None), udf.call(map(inputCol).attr) as map(outputCol))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.ml.example
package org.apache.spark.ml.classification

import org.apache.spark.ml._
import org.apache.spark.ml.param._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.ml.example
package org.apache.spark.ml.evaluation

import org.apache.spark.ml._
import org.apache.spark.ml.param._
Expand Down
42 changes: 0 additions & 42 deletions mllib/src/main/scala/org/apache/spark/ml/example/Tokenizer.scala

This file was deleted.

Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
package org.apache.spark.ml.example
package org.apache.spark.ml.feature

import org.apache.spark.ml.Transformer
import org.apache.spark.ml.param.{HasInputCol, HasOutputCol, IntParam, ParamMap}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.ml.example
package org.apache.spark.ml.feature

import org.apache.spark.ml._
import org.apache.spark.ml.param._
Expand All @@ -31,6 +31,10 @@ import org.apache.spark.sql.catalyst.expressions.Row
*/
trait StandardScalerParams extends Params with HasInputCol with HasOutputCol

/**
* Standardizes features by removing the mean and scaling to unit variance using column summary
* statistics on the samples in the training set.
*/
class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerParams {

def setInputCol(value: String): this.type = { set(inputCol, value); this }
Expand All @@ -50,6 +54,9 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP
}
}

/**
* Model fitted by [[StandardScaler]].
*/
class StandardScalerModel private[ml] (
override val parent: StandardScaler,
override val fittingParamMap: ParamMap,
Expand Down
Loading

0 comments on commit e246f29

Please sign in to comment.