You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by me...@apache.org on 2014/07/15 11:15:07 UTC

git commit: [SPARK-2477][MLlib] Using appendBias for adding intercept in GeneralizedLinearAlgorithm

Repository: spark
Updated Branches:
  refs/heads/master dd95abada -> 52beb20f7


[SPARK-2477][MLlib] Using appendBias for adding intercept in GeneralizedLinearAlgorithm

Instead of using prependOne currently in GeneralizedLinearAlgorithm, we would like to use appendBias for 1) keeping the indices of original training set unchanged by adding the intercept into the last element of vector and 2) using the same public API for consistently adding intercept.

Author: DB Tsai <db...@alpinenow.com>

Closes #1410 from dbtsai/SPARK-2477_intercept_with_appendBias and squashes the following commits:

011432c [DB Tsai] From Alpine Data Labs


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/52beb20f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/52beb20f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/52beb20f

Branch: refs/heads/master
Commit: 52beb20f7904e0333198b9b14619366ddf53ab85
Parents: dd95aba
Author: DB Tsai <db...@alpinenow.com>
Authored: Tue Jul 15 02:14:58 2014 -0700
Committer: Xiangrui Meng <me...@databricks.com>
Committed: Tue Jul 15 02:14:58 2014 -0700

----------------------------------------------------------------------
 .../regression/GeneralizedLinearAlgorithm.scala | 21 +++++---------------
 1 file changed, 5 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/52beb20f/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
index 8cca926..fe41863 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -17,13 +17,12 @@
 
 package org.apache.spark.mllib.regression
 
-import breeze.linalg.{DenseVector => BDV, SparseVector => BSV}
-
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.{Logging, SparkException}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.mllib.optimization._
 import org.apache.spark.mllib.linalg.{Vectors, Vector}
+import org.apache.spark.mllib.util.MLUtils._
 
 /**
  * :: DeveloperApi ::
@@ -124,16 +123,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
     run(input, initialWeights)
   }
 
-  /** Prepends one to the input vector. */
-  private def prependOne(vector: Vector): Vector = {
-    val vector1 = vector.toBreeze match {
-      case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv)
-      case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv)
-      case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
-    }
-    Vectors.fromBreeze(vector1)
-  }
-
   /**
    * Run the algorithm with the configured parameters on an input RDD
    * of LabeledPoint entries starting from the initial weights provided.
@@ -147,23 +136,23 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
 
     // Prepend an extra variable consisting of all 1.0's for the intercept.
     val data = if (addIntercept) {
-      input.map(labeledPoint => (labeledPoint.label, prependOne(labeledPoint.features)))
+      input.map(labeledPoint => (labeledPoint.label, appendBias(labeledPoint.features)))
     } else {
       input.map(labeledPoint => (labeledPoint.label, labeledPoint.features))
     }
 
     val initialWeightsWithIntercept = if (addIntercept) {
-      prependOne(initialWeights)
+      appendBias(initialWeights)
     } else {
       initialWeights
     }
 
     val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept)
 
-    val intercept = if (addIntercept) weightsWithIntercept(0) else 0.0
+    val intercept = if (addIntercept) weightsWithIntercept(weightsWithIntercept.size - 1) else 0.0
     val weights =
       if (addIntercept) {
-        Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size))
+        Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1))
       } else {
         weightsWithIntercept
       }