You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@mahout.apache.org by GitBox <gi...@apache.org> on 2021/01/21 16:24:01 UTC

[GitHub] [mahout] josehernandezsc opened a new pull request #413: MAHOUT-2136 [WIP] Brief Description of Changes

josehernandezsc opened a new pull request #413:
URL: https://github.com/apache/mahout/pull/413


   ### Purpose of PR:
   Mahout contribution: Implementation of Ridge Regression Algorithm by using closed form method.
   
   
   ### Important ToDos
   Please mark each with an "x"
   - [X] A JIRA ticket exists (if not, please create this first)[https://issues.apache.org/jira/browse/mahout/]
   - [X] Title of PR is "MAHOUT-XXXX Brief Description of Changes" where XXXX is the JIRA number.
   - [X] Created unit tests where appropriate
   - [X] Added licenses correct on newly added files
   - [ ] Assigned JIRA to self
   - [ ] Added documentation in scala docs/java docs, and to website
   - [X] Successfully built and ran all unit tests, verified that all tests pass locally.
   
   If all of these things aren't complete, but you still feel it is
   appropriate to open a PR, please add [WIP] after MAHOUT-XXXX before the
   descriptions- e.g. "MAHOUT-XXXX [WIP] Description of Change"
   
   Does this change break earlier versions?
   No
   
   Is this the beginning of a larger project for which a feature branch should be made?
   No


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo commented on pull request #413: MAHOUT-2136 [WIP] Brief Description of Changes

Posted by GitBox <gi...@apache.org>.
rawkintrevo commented on pull request #413:
URL: https://github.com/apache/mahout/pull/413#issuecomment-764768880


   Hey Jose- thanks for the contribution!
   
   I think you need to make sure to commit the files you updated, and remove the `mylogs.txt` file. At the moment, the logs are all you committed. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo commented on pull request #413: MAHOUT-2136 Ridge Regression

Posted by GitBox <gi...@apache.org>.
rawkintrevo commented on pull request #413:
URL: https://github.com/apache/mahout/pull/413#issuecomment-768643801


   @josehernandezsc you were right re: addIntercept- I'm a dummy.
   
   revert that and I think you're good to merge(!!!)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] josehernandezsc commented on a change in pull request #413: MAHOUT-2136 Ridge Regression

Posted by GitBox <gi...@apache.org>.
josehernandezsc commented on a change in pull request #413:
URL: https://github.com/apache/mahout/pull/413#discussion_r564966395



##########
File path: docker/getting-started/interpreter.json
##########
@@ -1,3 +1,20 @@
+#
+# 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.
+#

Review comment:
       I should have misplaced the comments in the interpreter.json file. I'm taking care of it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo merged pull request #413: MAHOUT-2136 Ridge Regression

Posted by GitBox <gi...@apache.org>.
rawkintrevo merged pull request #413:
URL: https://github.com/apache/mahout/pull/413


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo commented on a change in pull request #413: MAHOUT-2136 Ridge Regression

Posted by GitBox <gi...@apache.org>.
rawkintrevo commented on a change in pull request #413:
URL: https://github.com/apache/mahout/pull/413#discussion_r565706178



##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }

Review comment:
       you know what- you're probably right, its been a while since I looked at this part of the code base. go ahead and revert this change. Sorry about that -_-




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo commented on pull request #413: MAHOUT-2136 [WIP] Brief Description of Changes

Posted by GitBox <gi...@apache.org>.
rawkintrevo commented on pull request #413:
URL: https://github.com/apache/mahout/pull/413#issuecomment-764768880


   Hey Jose- thanks for the contribution!
   
   I think you need to make sure to commit the files you updated, and remove the `mylogs.txt` file. At the moment, the logs are all you committed. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo commented on a change in pull request #413: MAHOUT-2136 [WIP] Ridge Regression

Posted by GitBox <gi...@apache.org>.
rawkintrevo commented on a change in pull request #413:
URL: https://github.com/apache/mahout/pull/413#discussion_r564188580



##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }
+
+  def fit(drmFeatures: DrmLike[K], drmTarget: DrmLike[K], hyperparameters: (Symbol, Any)*): RidgeRegressionModel[K] = {//lambda: Double = 0.1) = {
+
+
+    var model = new RidgeRegressionModel[K]
+    setStandardHyperparameters(hyperparameters.toMap)
+
+    var X = drmFeatures
+
+    if (addIntercept) {
+      X = X cbind 1
+    }
+
+
+    val XTX = (X.t %*% X).collect
+    val drmXtXinv = solve(XTX)
+    val XTy = X.t %*% drmTarget
+    val reg = diag(lambda, XTX.ncol)
+
+    val sol = solve(XTX.plus(reg), XTy)
+
+    model.beta = sol(::, 0)
+
+    this.modelPostprocessing(model, X, drmTarget, drmXtXinv)
+
+    //model.lambda = lambda

Review comment:
       delete this line

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context

Review comment:
       drop comment

##########
File path: docker/getting-started/interpreter.json
##########
@@ -1,3 +1,20 @@
+#
+# 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.
+#

Review comment:
       Can you put comments in json files? I'm not 100% sure this is appropriate (?) I really don't know, if it breaks something we can always revert it. 

##########
File path: website/docs/latest/algorithms/regression/ridge.md
##########
@@ -0,0 +1,55 @@
+---
+layout: doc-page
+title: Ridge Regression
+---
+
+### About
+
+The `Ridge` regressor in Mahout implements a _closed-form_ solution to Ridge Regression (https://en.wikipedia.org/wiki/Tikhonov_regularization).
+
+

Review comment:
       If you have some time I might explain a little more about what Ridge regression is / when its appropriate to use, etc. This is good, and you have a wiki link which is also nice, I'm just picking nits. 

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))

Review comment:
       drop this comment

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag

Review comment:
       drop comment

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _

Review comment:
       drop comment

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }

Review comment:
       I think you need to include 'addIntercept here

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+

Review comment:
       a lot of white space here...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] rawkintrevo commented on a change in pull request #413: MAHOUT-2136 [WIP] Ridge Regression

Posted by GitBox <gi...@apache.org>.
rawkintrevo commented on a change in pull request #413:
URL: https://github.com/apache/mahout/pull/413#discussion_r564188580



##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }
+
+  def fit(drmFeatures: DrmLike[K], drmTarget: DrmLike[K], hyperparameters: (Symbol, Any)*): RidgeRegressionModel[K] = {//lambda: Double = 0.1) = {
+
+
+    var model = new RidgeRegressionModel[K]
+    setStandardHyperparameters(hyperparameters.toMap)
+
+    var X = drmFeatures
+
+    if (addIntercept) {
+      X = X cbind 1
+    }
+
+
+    val XTX = (X.t %*% X).collect
+    val drmXtXinv = solve(XTX)
+    val XTy = X.t %*% drmTarget
+    val reg = diag(lambda, XTX.ncol)
+
+    val sol = solve(XTX.plus(reg), XTy)
+
+    model.beta = sol(::, 0)
+
+    this.modelPostprocessing(model, X, drmTarget, drmXtXinv)
+
+    //model.lambda = lambda

Review comment:
       delete this line

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context

Review comment:
       drop comment

##########
File path: docker/getting-started/interpreter.json
##########
@@ -1,3 +1,20 @@
+#
+# 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.
+#

Review comment:
       Can you put comments in json files? I'm not 100% sure this is appropriate (?) I really don't know, if it breaks something we can always revert it. 

##########
File path: website/docs/latest/algorithms/regression/ridge.md
##########
@@ -0,0 +1,55 @@
+---
+layout: doc-page
+title: Ridge Regression
+---
+
+### About
+
+The `Ridge` regressor in Mahout implements a _closed-form_ solution to Ridge Regression (https://en.wikipedia.org/wiki/Tikhonov_regularization).
+
+

Review comment:
       If you have some time I might explain a little more about what Ridge regression is / when its appropriate to use, etc. This is good, and you have a wiki link which is also nice, I'm just picking nits. 

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))

Review comment:
       drop this comment

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag

Review comment:
       drop comment

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _

Review comment:
       drop comment

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }

Review comment:
       I think you need to include 'addIntercept here

##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+

Review comment:
       a lot of white space here...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] josehernandezsc commented on a change in pull request #413: MAHOUT-2136 Ridge Regression

Posted by GitBox <gi...@apache.org>.
josehernandezsc commented on a change in pull request #413:
URL: https://github.com/apache/mahout/pull/413#discussion_r565729781



##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }

Review comment:
       Sure, no problem!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [mahout] josehernandezsc commented on a change in pull request #413: MAHOUT-2136 Ridge Regression

Posted by GitBox <gi...@apache.org>.
josehernandezsc commented on a change in pull request #413:
URL: https://github.com/apache/mahout/pull/413#discussion_r564962509



##########
File path: core/src/main/scala/org/apache/mahout/math/algorithms/regression/RidgeRegressionModel.scala
##########
@@ -0,0 +1,96 @@
+/**
+  * 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.mahout.math.algorithms.regression
+
+import org.apache.mahout.math._
+import org.apache.mahout.math.scalabindings._
+import org.apache.mahout.math.drm._
+import org.apache.mahout.math.scalabindings.RLikeOps._
+import org.apache.mahout.math.drm.RLikeDrmOps._
+
+
+
+
+class RidgeRegressionModel[K] extends LinearRegressorModel[K]{
+  //var lambda: Double = _
+  def predict(drmPredictors: DrmLike[K]): DrmLike[K] = {
+
+
+    //implicit val ktag =  drmPredictors.keyClassTag
+
+    //implicit val ctx = drmPredictors.context
+
+    //val bcGuess = drmBroadcast(dvec(lambda))
+
+
+
+    var X = drmPredictors
+    if (addIntercept) {
+      X = X cbind 1
+    }
+    X %*% beta
+
+  }
+
+
+
+
+}
+
+
+
+
+class RidgeRegression[K] extends LinearRegressorFitter[K] {
+
+  var lambda: Double = _
+
+  override def setStandardHyperparameters(hyperparameters: Map[Symbol, Any] = Map('foo -> None)): Unit = {
+    lambda = hyperparameters.asInstanceOf[Map[Symbol, Double]].getOrElse('lambda,1.0)
+  }

Review comment:
       I'm adding 'addIntercept because of the override method used in the setStandardHyperparameters function. Normally I'm almost sure it would inherit from the LinearRegressor object. All tests came out successful after the change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org