Skip to content

Commit

Permalink
add predictOnValues to StreamingLR and fix predictOn
Browse files Browse the repository at this point in the history
  • Loading branch information
mengxr committed Aug 18, 2014
1 parent d1d0ee4 commit 0925efa
Show file tree
Hide file tree
Showing 2 changed files with 24 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,10 +59,10 @@ object StreamingLinearRegression {
val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse)

val model = new StreamingLinearRegressionWithSGD()
.setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0)))
.setInitialWeights(Vectors.zeros(args(3).toInt))

model.trainOn(trainingData)
model.predictOn(testData).print()
model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print()

ssc.start()
ssc.awaitTermination()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,9 @@

package org.apache.spark.mllib.regression

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.streaming.dstream.DStream

/**
Expand Down Expand Up @@ -92,15 +93,30 @@ abstract class StreamingLinearAlgorithm[
/**
* Use the model to make predictions on batches of data from a DStream
*
* @param data DStream containing labeled data
* @param data DStream containing feature vectors
* @return DStream containing predictions
*/
def predictOn(data: DStream[LabeledPoint]): DStream[Double] = {
def predictOn(data: DStream[Vector]): DStream[Double] = {
if (Option(model.weights) == None) {
logError("Initial weights must be set before starting prediction")
throw new IllegalArgumentException
val msg = "Initial weights must be set before starting prediction"
logError(msg)
throw new IllegalArgumentException(msg)
}
data.map(x => model.predict(x.features))
data.map(model.predict)
}

/**
* Use the model to make predictions on the values of a DStream and carry over its keys.
* @param data DStream containing feature vectors
* @tparam K key type
* @return DStream containing the input keys and the predictions as values
*/
def predictOnValues[K](data: DStream[(K, Vector)]): DStream[(K, Double)] = {
if (Option(model.weights) == None) {
val msg = "Initial weights must be set before starting prediction"
logError(msg)
throw new IllegalArgumentException(msg)
}
data.mapPartitions(_.map(x => (x._1, model.predict(x._2))), preservePartitioning = true)
}
}

0 comments on commit 0925efa

Please sign in to comment.