Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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,12 @@

package org.apache.spark.mllib.regression

import org.apache.spark.annotation.DeveloperApi
import scala.reflect.ClassTag

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

/**
Expand Down Expand Up @@ -92,15 +96,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: ClassTag](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.mapValues(model.predict)
}
}