You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by manishamde <gi...@git.apache.org> on 2014/10/01 03:37:02 UTC

[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

GitHub user manishamde opened a pull request:

    https://github.com/apache/spark/pull/2607

    [MLLIB] [WIP] SPARK-1547: Adding Gradient Boosting to MLlib

    Given the popular demand for gradient boosting and AdaBoost in MLlib, I am creating a WIP branch for early feedback on gradient boosting with AdaBoost to follow soon after this PR is accepted. This is based on work done along with @hirakendu that was pending due to decision tree optimizations and random forests work.
    
    Ideally, boosting algorithms should work with any base learners.  This will soon be possible once the MLlib API is finalized -- we want to ensure we use a consistent interface for the underlying base learners. In the meantime, this PR uses decision trees as base learners for the gradient boosting algorithm. The current PR allows "pluggable" loss functions and provides least squares error and least absolute error by default.
    
    Here is the remaining task list:
    - [ ] Stochastic gradient boosting support – Re-use the BaggedPoint approach used for RandomForest.
    - [ ] BaggedRDD caching -- Avoid repeating feature to bin mapping for each tree estimator. Will require minor refactoring of RandomForest code.
    - [ ] Checkpointing – This approach will avoid long lineage chains. Need to conduct experiments to verify good default settings.
    - [ ] Unit Tests – I have performed some basic tests but I need to add them as unit tests.
    - [ ] Create public APIs
    - [ ] Tests on multiple cluster sizes and datasets – require help from the community on this front.
    
    Note: Classification is currently not supported by this PR since it requires discussion on the best way to support "deviance" as a loss function.
    
    cc: @jkbradley @hirakendu @mengxr @etrain @atalwalkar @chouqin

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/manishamde/spark gbt

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/2607.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2607
    
----
commit 0ae1c0a77c9de22dd1ff50ad1e4c7b8a691aac38
Author: Manish Amde <ma...@gmail.com>
Date:   2014-09-28T03:32:22Z

    basic gradient boosting code from earlier branches

commit 55385216ff2d0a470ae783017d434d850762441f
Author: Manish Amde <ma...@gmail.com>
Date:   2014-09-28T04:32:31Z

    disable checkpointing for now

commit 6251fd56388703d9b9450980a27cf9a9a98e750d
Author: Manish Amde <ma...@gmail.com>
Date:   2014-10-01T00:22:26Z

    modified method name

commit cdceeef09822145af2620921a94c37384d3f64c7
Author: Manish Amde <ma...@gmail.com>
Date:   2014-10-01T01:04:02Z

    added documentation

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58991829
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21716/consoleFull) for   PR 2607 at commit [`1f47941`](https://github.com/apache/spark/commit/1f47941820737598831a2ba932d0319148b1598c).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61221341
  
    @jkbradley I cleaned up the public API based on our discussion. Going with a nested structure where we have to specify the weak learner parameters separately is cleaner but it puts the onus on us to write very good documentation.
    
    I am tempted to keep AbsoluteError and LogLoss as is with the appropriate caveats in the documentation. A regression tree with mean prediction at the terminal nodes it not the best approximation (as pointed out by the TreeBoost paper) but it's not a bad one either. After all, we are just making approximations of the gradient at each step. Moreover, other weak learning algorithms (for example LR) will be hard to tailor towards each specific loss function. Thoughts? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049191
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,42 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm
    + */
    +trait Loss extends Serializable {
    --- End diff --
    
    Can this also include a "loss" or "compute" method?  That would allow tracking the actual objective in boosting (instead of just MSE as is done now).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495219
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm.
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation.
    +   * @param model Model of the weak learner.
    +   * @param point Instance of the training dataset.
    +   * @param learningRate Learning rate parameter for regularization.
    +   * @return Loss gradient.
    +   */
    +  @DeveloperApi
    +  def lossGradient(
    --- End diff --
    
    Could this please be renamed to "gradient" so it is less repetitive to call loss.lossGradient?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61068865
  
      [Test build #22533 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22533/consoleFull) for   PR 2607 at commit [`1c40c33`](https://github.com/apache/spark/commit/1c40c33e73d68edaa14b5573c5cdef2b591c6419).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496241
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.mllib.tree.impl
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.tree.EnsembleTestHelper
    +import org.apache.spark.mllib.util.LocalSparkContext
    +
    +/**
    + * Test suite for [[BaggedPoint]].
    + */
    +class BaggedPointSuite extends FunSuite with LocalSparkContext  {
    +
    +  test("BaggedPoint RDD: without subsampling") {
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false)
    +    baggedRDD.collect().foreach { baggedPoint =>
    +      assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 1.0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample * (1 - subsample)))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/2607


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049194
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,42 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation
    +   * @param model Model of the weak learner
    +   * @param point Instance of the training dataset
    +   * @param learningRate Learning rate parameter for regularization
    +   * @return Loss gradient
    +   */
    +  @DeveloperApi
    +  def lossGradient(
    --- End diff --
    
    Can we name this "gradient"?  (Calling Loss.lossGradient seems redundant)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19497888
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    --- End diff --
    
    Use same name: "checkpointingPeriod" or "checkpointPeriod"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496253
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala ---
    @@ -0,0 +1,208 @@
    +/*
    + * 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.mllib.tree
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy}
    +import org.apache.spark.mllib.tree.impurity.{Variance, Gini}
    +import org.apache.spark.mllib.tree.loss.{SquaredError, LogLoss}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +
    +import org.apache.spark.mllib.util.LocalSparkContext
    +
    +/**
    + * Test suite for [[GradientBoosting]].
    + */
    +class GradientBoostingSuite extends FunSuite with LocalSparkContext {
    +
    +  test("Binary classification with continuous features:" +
    +    " comparing DecisionTree vs. GradientBoosting (numEstimators = 1)") {
    +
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numEstimators = 1
    +
    +    val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +    val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val dt = DecisionTree.train(remappedInput, treeStrategy)
    +
    +    val boostingStrategy = new BoostingStrategy(algo = Classification,
    +      numEstimators = numEstimators, loss = LogLoss, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy)
    +    assert(gbt.baseLearners.size === 1)
    +    val gbtTree = gbt.baseLearners(0)
    +
    +
    +    EnsembleTestHelper.validateClassifier(gbt, arr, 0.9)
    +
    +    // Make sure trees are the same.
    +    assert(gbtTree.toString == dt.toString)
    +  }
    +
    +  test("Binary classification with continuous features:" +
    +    " comparing DecisionTree vs. GradientBoosting (numEstimators = 10)") {
    +
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numEstimators = 10
    +
    +    val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +    val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val dt = DecisionTree.train(remappedInput, treeStrategy)
    +
    +    val boostingStrategy = new BoostingStrategy(algo = Classification,
    +      numEstimators = numEstimators, loss = LogLoss, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy)
    +    assert(gbt.baseLearners.size === 10)
    +    val gbtTree = gbt.baseLearners(0)
    +
    +
    +    EnsembleTestHelper.validateClassifier(gbt, arr, 0.9)
    +
    +    // Make sure trees are the same.
    +    assert(gbtTree.toString == dt.toString)
    +  }
    +
    +  test("Binary classification with continuous features:" +
    +    " Stochastic GradientBoosting (numEstimators = 10, learning rate = 0.9, subsample = 0.75)") {
    +
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numEstimators = 10
    +
    +    val boostingStrategy = new BoostingStrategy(algo = Classification,
    +      numEstimators = numEstimators, loss = LogLoss, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo,
    +      subsample = 0.75)
    +
    +    val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy)
    +    assert(gbt.baseLearners.size === 10)
    +
    +    EnsembleTestHelper.validateClassifier(gbt, arr, 0.9)
    +
    +  }
    +
    +
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58987769
  
    @jkbradley I already have support for a parameter class called ```BoostingStrategy```. I also have support for a shorter argument list when the user wants to specify only the minimum number of parameters to train. However, the long argument list is useful when an advanced user wants access to all the options. Also, in the other scenario, we end up having a long parameter to list to call the constructor of the parameter class.
    
    Ideally, a user should not have to worry about using algorithm specific parameters classes (which we mark as Experimental) to access the algorithms. I can remove some tree-specific options to fit the limit of 10 at the risk of making it harder for users to set advanced options. I think the limit of 10 is arbitrary for external facing API especially since Scala and Python support named named parameters with default values (unfortunately Java does not). Having said that, we can separate boosting and underlying algo parameters after standard MLlib API work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60512854
  
      [Test build #22241 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22241/consoleFull) for   PR 2607 at commit [`d971f73`](https://github.com/apache/spark/commit/d971f73ef330ab33edeae5a4514d5246196d55f1).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58958069
  
      [QA tests have started](consoleFull) for   PR 2607 at commit [`2cb1258`](https://github.com/apache/spark/commit/2cb12582e956a6a95920bbaa5a5c9e5ff4387e69).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60509450
  
      [Test build #22239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22239/consoleFull) for   PR 2607 at commit [`fee06d3`](https://github.com/apache/spark/commit/fee06d3aa4bcc6b163fcd468c47163d838537c0c).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19497891
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    --- End diff --
    
    Use "GradientBoosting$#trainRegressor" instead of "GradientBoosting#trainRegressor" so doc processor handles object correctly (using Java reflection syntax).  Please update elsewhere too


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58772988
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21653/consoleFull) for   PR 2607 at commit [`6dd4dd8`](https://github.com/apache/spark/commit/6dd4dd82ac6d42be0edb3f7498b300cc121a8021).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59589703
  
    @manishamde  Sorry for the delay; the code is looking good!  I made some small comments inline.  My main overall comment is about specifying parameters.  How would it be if we started mimicking the coming API update (as much as possible)?  Parameter specification would work as follows:
    
    In DecisionTree, add a static “defaultParams” method so users can construct a tree.Strategy instance without having to worry about importing Strategy (and remembering its name).  Likewise for GradientBoosting.
    
    Change GradientBoostingStrategy to store tree params in a field weakLearnerParams: tree.Strategy
    
    Here’s the use pattern I envision:
    
    val treeParams = DecisionTree.defaultParams()
    treeParams.maxDepth = ...
    val boostingParams = GradientBoosting.defaultParams()
    boostingParams.weakLearnerParams = treeParams
    val model = GradientBoosting.train(myData, boostingParams)
    
    This API should work for Scala and Python right away.  (Though a Python API can be another PR.)
    
    For Java, this API should almost work; I believe the only issue will be setting fields which take special types (e.g., quantileCalculationStrategy and categoricalFeaturesInfo).  For those, there is a nice annotation you can use which will automatically add getParamName and setParamName methods for Java users to call, and you can override them as needed.  For the special params like categoricalFeaturesInfo, you can overload them with versions which take Java-friendly types (such as a Java map for categoricalFeaturesInfo and a string for quantileCalculationStrategy).  Here’s the BeanProperty doc:
    [http://www.scala-lang.org/api/current/scala/beans/BeanProperty.html]
    
    Does that sound reasonable?
    
    Let me know when it’s ready for another pass and for testing.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773337
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21658/consoleFull) for   PR 2607 at commit [`3b8ffc0`](https://github.com/apache/spark/commit/3b8ffc00e9854ca323f0c8772784bf1337eec562).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60821283
  
    @jkbradley Your understanding is correct. Sorry for not mentioning it explicitly on the JIRA/PR earlier. 
    
    Yes, calculating median, etc. for terminal region predictions in trees will be hard for the distributed decision tree. I can add a warning mentioning in what way our implementation is different from Friedman's algorithm. I was planning to point it out in the documentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58971002
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21707/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19499327
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm.
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation.
    +   * @param model Model of the weak learner.
    +   * @param point Instance of the training dataset.
    +   * @param learningRate Learning rate parameter for regularization.
    +   * @return Loss gradient.
    --- End diff --
    
    Sounds good. Agree.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495244
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.mllib.tree.impl
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.tree.EnsembleTestHelper
    +import org.apache.spark.mllib.util.LocalSparkContext
    +
    +/**
    + * Test suite for [[BaggedPoint]].
    + */
    +class BaggedPointSuite extends FunSuite with LocalSparkContext  {
    +
    +  test("BaggedPoint RDD: without subsampling") {
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false)
    +    baggedRDD.collect().foreach { baggedPoint =>
    +      assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 1.0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample * (1 - subsample)))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    --- End diff --
    
    extra spaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19639714
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,412 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression and binary classification problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +    timer.start("total")
    +    timer.start("init")
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstTreeModel = new DecisionTree(strategy).train(data)
    +    baseLearners(0) = firstTreeModel
    +    baseLearnerWeights(0) = 1.0
    --- End diff --
    
    I think the learning rate is applied after the first model.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58118620
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21353/Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58118617
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21353/consoleFull) for   PR 2607 at commit [`78ed452`](https://github.com/apache/spark/commit/78ed452a522a651d54e4cb5d5381a2ee5c559270).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19638835
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala ---
    @@ -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.mllib.tree
    +
    +import org.apache.spark.mllib.linalg.Vectors
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel
    +import org.apache.spark.util.StatCounter
    +
    +import scala.collection.mutable
    +
    +object EnsembleTestHelper {
    +
    +  /**
    +   * Aggregates all values in data, and tests whether the empirical mean and stddev are within
    +   * epsilon of the expected values.
    +   * @param data  Every element of the data should be an i.i.d. sample from some distribution.
    +   */
    +  def testRandomArrays(
    +      data: Array[Array[Double]],
    +      numCols: Int,
    +      expectedMean: Double,
    +      expectedStddev: Double,
    +      epsilon: Double) {
    +    val values = new mutable.ArrayBuffer[Double]()
    +    data.foreach { row =>
    +      assert(row.size == numCols)
    +      values ++= row
    +    }
    +    val stats = new StatCounter(values)
    +    assert(math.abs(stats.mean - expectedMean) < epsilon)
    +    assert(math.abs(stats.stdev - expectedStddev) < epsilon)
    +  }
    +
    +  def validateClassifier(
    +      model: WeightedEnsembleModel,
    +      input: Seq[LabeledPoint],
    +      requiredAccuracy: Double) {
    +    val predictions = input.map(x => model.predict(x.features))
    +    val numOffPredictions = predictions.zip(input).count { case (prediction, expected) =>
    +      prediction != expected.label
    +    }
    +    val accuracy = (input.length - numOffPredictions).toDouble / input.length
    +    assert(accuracy >= requiredAccuracy,
    +      s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.")
    +  }
    +
    +  def validateRegressor(
    +      model: WeightedEnsembleModel,
    +      input: Seq[LabeledPoint],
    +      requiredMSE: Double) {
    +    val predictions = input.map(x => model.predict(x.features))
    +    val squaredError = predictions.zip(input).map { case (prediction, expected) =>
    +      val err = prediction - expected.label
    +      err * err
    +    }.sum
    +    val mse = squaredError / input.length
    +    assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.")
    +  }
    +
    +  def generateOrderedLabeledPoints(numFeatures: Int, numInstances: Int): Array[LabeledPoint] = {
    +    val arr = new Array[LabeledPoint](numInstances)
    +    for (i <- 0 until numInstances) {
    +      val label = if (i < numInstances / 10) {
    +        0.0
    +      } else if (i < numInstances / 2) {
    +        1.0
    +      } else if (i < numInstances * 0.9) {
    +        0.0
    +      } else {
    +        1.0
    +      }
    +      val features = Array.fill[Double](numFeatures)(i.toDouble)
    +      arr(i) = new LabeledPoint(label, Vectors.dense(features))
    +    }
    +    arr
    +  }
    +
    +}
    --- End diff --
    
    sure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60821510
  
    Great, that sounds reasonable.  I believe we could do it eventually: since the trees won't be too deep in many cases, the sufficient stats to pass around might be manageable.  Future work!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60511234
  
      [Test build #22241 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22241/consoleFull) for   PR 2607 at commit [`d971f73`](https://github.com/apache/spark/commit/d971f73ef330ab33edeae5a4514d5246196d55f1).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773258
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21654/consoleFull) for   PR 2607 at commit [`2fbc9c7`](https://github.com/apache/spark/commit/2fbc9c74885617ffc61c2fa9add1148e28acaf91).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495249
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.mllib.tree.impl
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.tree.EnsembleTestHelper
    +import org.apache.spark.mllib.util.LocalSparkContext
    +
    +/**
    + * Test suite for [[BaggedPoint]].
    + */
    +class BaggedPointSuite extends FunSuite with LocalSparkContext  {
    +
    +  test("BaggedPoint RDD: without subsampling") {
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false)
    +    baggedRDD.collect().foreach { baggedPoint =>
    +      assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 1.0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample * (1 - subsample)))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +
    +
    +}
    --- End diff --
    
    newline needed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773981
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21658/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by codedeft <gi...@git.apache.org>.
Github user codedeft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19567364
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity}
     import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
     import org.apache.spark.mllib.tree.configuration.Algo._
    -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    --- End diff --
    
    Yea, I guess from the design perspective, it's tempting to unify these under the same umbrella.
    
    IMO, RandomForest is *mostly* a specific instance of a generic ensemble model, so this makes sense.
    
    However, I think that boosted models have some specific things about them due to their sequential nature (as opposed to parallel nature of RandomForest). E.g., if you have 1000 models, you can potentially predict based on the *first* 100 models whereas with RandomForest you can pick any 100. You also have to do overfitting/underfitting analyses on boosted models sequentially, etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61075779
  
      [Test build #22532 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22532/consoleFull) for   PR 2607 at commit [`e33ab61`](https://github.com/apache/spark/commit/e33ab61bfc6e11de6f8a72368de660b90aea1345).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61075232
  
      [Test build #22531 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22531/consoleFull) for   PR 2607 at commit [`035a2ed`](https://github.com/apache/spark/commit/035a2ed6bb09910a3e8a6593b3276b742cf7b7d2).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58772394
  
    @mengxr The test failures seem to be due to lack of permission. Could you please enable it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58994169
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21716/consoleFull) for   PR 2607 at commit [`1f47941`](https://github.com/apache/spark/commit/1f47941820737598831a2ba932d0319148b1598c).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61078179
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22535/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19066152
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,42 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation
    +   * @param model Model of the weak learner
    +   * @param point Instance of the training dataset
    +   * @param learningRate Learning rate parameter for regularization
    +   * @return Loss gradient
    +   */
    +  @DeveloperApi
    +  def lossGradient(
    --- End diff --
    
    negativeGradient might be better. What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58972792
  
    While I agree with this style check for internal methods, I am not sure this is a good style check when the algorithm parameters can be greater than 10.
    
    ```message=The number of parameters should not exceed 10```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58991200
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21714/consoleFull) for   PR 2607 at commit [`5b67102`](https://github.com/apache/spark/commit/5b67102bdd5071365d948f90464bc6bbc67d2089).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496210
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.mllib.tree.configuration
    +
    +import org.apache.spark.annotation.DeveloperApi
    +
    +/**
    + * :: Experimental ::
    + * Enum to select ensemble combining strategy for base learners
    + */
    +@DeveloperApi
    +object EnsembleCombiningStrategy extends Enumeration {
    --- End diff --
    
    Why are sum and average the same? In RF you average predictions and in GBT you add predictions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61327135
  
    Thanks. Sounds good to me. 
    
    I tried to use the builder pattern to help for the Java use case but I guess we can handle it separately. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-57408875
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21079/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61075241
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22531/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60536963
  
      [Test build #22263 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22263/consoleFull) for   PR 2607 at commit [`781542a`](https://github.com/apache/spark/commit/781542a29ffa480c54a8aff67f5344ac698613fb).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58956426
  
    @jkbradley Looking into it. I don't see the same error when I compile locally.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19570259
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    Correct. That's the big disadvantage of not using the internal format. It won't affect other algos as much since there is no discretization.
    
    We have a few options:
    1. Keep the implementation as is and inform the user about memory requirements.
    2. Persisting RDD[TreePoint] is essential since we perform multiple passes on it during each tree construction and reading RDD[LabeledPoint] from disk every time.
    3. Persisting RDD[LabeledPoint] and not caching RDD[TreePoint] during tree construction leading to repeated LabeledPoint -> TreePoint conversions for each NodeGroup.
    
    Thoughts? cc: @jkbradley 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58772868
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21655/consoleFull) for   PR 2607 at commit [`bdca43a`](https://github.com/apache/spark/commit/bdca43a4679f194c22209733852515cfa09bf407).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58972200
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21708/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773732
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21657/consoleFull) for   PR 2607 at commit [`8e10c63`](https://github.com/apache/spark/commit/8e10c6364cf9ac0fffbfc63e1234a8e44c77a640).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496224
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.mllib.tree.impl
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.tree.EnsembleTestHelper
    +import org.apache.spark.mllib.util.LocalSparkContext
    +
    +/**
    + * Test suite for [[BaggedPoint]].
    + */
    +class BaggedPointSuite extends FunSuite with LocalSparkContext  {
    +
    +  test("BaggedPoint RDD: without subsampling") {
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false)
    +    baggedRDD.collect().foreach { baggedPoint =>
    +      assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 1.0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling with replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 1.0)") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling without replacement (fraction = 0.5)") {
    +    val numSubsamples = 100
    +    val subsample = 0.5
    +    val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample * (1 - subsample)))
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean,
    +        expectedStddev, epsilon = 0.01)
    +    }
    +  }
    +
    +
    +
    +}
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496095
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -46,20 +47,63 @@ private[tree] object BaggedPoint {
        * Convert an input dataset into its BaggedPoint representation,
        * choosing subsample counts for each instance.
        * Each subsample has the same number of instances as the original dataset,
    -   * and is created by subsampling with replacement.
    -   * @param input     Input dataset.
    -   * @param numSubsamples  Number of subsamples of this RDD to take.
    -   * @param seed   Random seed.
    -   * @return  BaggedPoint dataset representation
    +   * and is created by subsampling without replacement.
    +   * @param input Input dataset.
    +   * @param subsample Fraction of the training data used for learning decision tree.
    +   * @param numSubsamples Number of subsamples of this RDD to take.
    +   * @param withReplacement Sampling with/without replacement.
    +   * @param seed Random seed.
    +   * @return BaggedPoint dataset representation.
        */
    -  def convertToBaggedRDD[Datum](
    +  def convertToBaggedRDD[Datum] (
           input: RDD[Datum],
    +      subsample: Double,
    --- End diff --
    
    Agree. Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60512165
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22239/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19638821
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val weakHypotheses: Array[DecisionTreeModel],
    +    val weakHypothesisWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numWeakHypotheses > 0, s"WeightedEnsembleModel cannot be created without weakHypotheses" +
    +    s". Number of weakHypotheses = $weakHypotheses")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = weakHypotheses.map(learner => learner.predict(features))
    +    if (numWeakHypotheses == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numWeakHypotheses) {
    +        prediction += weakHypothesisWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    algo match {
    +      case Regression => predictRaw(features)
    +      case Classification => {
    +        // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info.
    +        if (predictRaw(features) > 0 ) 1.0 else 0.0
    +      }
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point.
    +   *
    +   * @param features array representing a single data point
    +   * @return Double prediction from the trained model
    +   */
    +  def predictByAveraging(features: Vector): Double = {
    +    algo match {
    +      case Classification =>
    +        val predictionToCount = new mutable.HashMap[Int, Int]()
    +        weakHypotheses.foreach { learner =>
    +          val prediction = learner.predict(features).toInt
    +          predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1
    +        }
    +        predictionToCount.maxBy(_._2)._1
    +      case Regression =>
    +        weakHypotheses.map(_.predict(features)).sum / weakHypotheses.size
    +    }
    +  }
    +
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  def predict(features: Vector): Double = {
    +    combiningStrategy match {
    +      case Sum => predictBySumming(features)
    +      case Average => predictByAveraging(features)
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown combining parameter: $combiningStrategy.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for the given data set.
    +   *
    +   * @param features RDD representing data points to be predicted
    +   * @return RDD[Double] where each entry contains the corresponding prediction
    +   */
    +  def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x))
    +
    +  /**
    +   * Print full model.
    +   */
    +  override def toString: String = {
    +    val header = algo match {
    +      case Classification =>
    +        s"WeightedEnsembleModel classifier with $numWeakHypotheses trees\n"
    +      case Regression =>
    +        s"WeightedEnsembleModel regressor with $numWeakHypotheses trees\n"
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +    header + weakHypotheses.zipWithIndex.map { case (learner, treeIndex) =>
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61072219
  
      [Test build #22536 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22536/consoleFull) for   PR 2607 at commit [`b4c1318`](https://github.com/apache/spark/commit/b4c13188de93f4e337bb82405de5969ae22cbdb9).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61077350
  
      [Test build #22533 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22533/consoleFull) for   PR 2607 at commit [`1c40c33`](https://github.com/apache/spark/commit/1c40c33e73d68edaa14b5573c5cdef2b591c6419).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61353864
  
    I've merged this into master. Thanks @manishamde for contributing and @codedeft and @jkbradley for review!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495252
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.mllib.tree.configuration
    +
    +import org.apache.spark.annotation.DeveloperApi
    +
    +/**
    + * :: Experimental ::
    + * Enum to select ensemble combining strategy for base learners
    + */
    +@DeveloperApi
    +object EnsembleCombiningStrategy extends Enumeration {
    --- End diff --
    
    I think this strategy option would be useful at some point, but not yet.
    * sum and average are essentially the same thing
    * Eventually, when we support options such as median, this could be nice to add
    Remove for now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60512164
  
    **[Test build #22239 timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22239/consoleFull)**     for PR 2607 at commit [`fee06d3`](https://github.com/apache/spark/commit/fee06d3aa4bcc6b163fcd468c47163d838537c0c)     after a configured wait of `120m`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049179
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -47,20 +48,61 @@ private[tree] object BaggedPoint {
        * Convert an input dataset into its BaggedPoint representation,
        * choosing subsample counts for each instance.
        * Each subsample has the same number of instances as the original dataset,
    -   * and is created by subsampling with replacement.
    +   * and is created by subsampling without replacement.
        * @param input     Input dataset.
    +   * @param subsample Fraction of the training data used for learning decision tree.
        * @param numSubsamples  Number of subsamples of this RDD to take.
    -   * @param seed   Random seed.
    +   * @param withReplacement Sampling with/without replacement.
        * @return  BaggedPoint dataset representation
        */
       def convertToBaggedRDD[Datum](
           input: RDD[Datum],
    +      subsample: Double,
           numSubsamples: Int,
    -      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    +      withReplacement: Boolean): RDD[BaggedPoint[Datum]] = {
    +    if (withReplacement) {
    +      convertToBaggedRDDSamplingWithReplacement(input, subsample, numSubsamples)
    +    } else {
    +      if (numSubsamples == 1 && subsample == 1.0) {
    +        convertToBaggedRDDWithoutSampling(input)
    +      } else {
    +        convertToBaggedRDDSamplingWithoutReplacement(input, subsample, numSubsamples)
    +      }
    +    }
    +  }
    +
    +  private[tree] def convertToBaggedRDDSamplingWithoutReplacement[Datum](
    --- End diff --
    
    space before (
    (same issue elsewhere too)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58403773
  
    Sounds good!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61174268
  
    @jkbradley Thanks for the confirmation! I will now proceed to finish the rest of the tasks -- should be straightforward.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59677818
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21902/consoleFull) for   PR 2607 at commit [`9366b8f`](https://github.com/apache/spark/commit/9366b8f52a31a10cab999f63872b248bddfeaffe).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59697993
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21908/consoleFull) for   PR 2607 at commit [`2ae97b7`](https://github.com/apache/spark/commit/2ae97b74ccc0e7fc3f34d435264768a1403a7a0c).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], strategy: BoostingStrategy)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495231
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val baseLearners: Array[DecisionTreeModel],
    --- End diff --
    
    Please switch from "learner" to "hypothesis" since these are hypotheses produced by the learner.  I'd recommend:
    * baseLearners -> weakHypotheses
    * baseLearnerWeights -> weakHypothesisWeights


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58231643
  
    @manishamde  About the 2 caching options, I agree with your decision to do (1) first.  It would be nice to try (2) later on (another PR?), but I don't think it is too high-priority.  Perhaps we can eventually have learning algs provide convertDatasetToInternalFormat() and predictUsingInternalFormat() methods (with less verbose names), once the standard API is in place.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by codedeft <gi...@git.apache.org>.
Github user codedeft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19563516
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    It seems to me that this will result in repetitive sampling/ re-discretization and etc. of the entire data set every iteration. Additionally, repersisting the entire dataset seems very expensive, in particular if the dataset (LabeledPoint) is initially coming from the disk.
    
    I think that the optimal thing to do is:
    1. Discretize the features and persist the entire discretized features only once.
    2. Calculate the new labels after each iteration, and create a separate RDD of these new labels, and persist them.
    3. zip the new labels with the discretized features and reuse the DecisionTree's regression logic.
    
    This will require some modifications of internal DecisionTree.train but it seems to me the better thing to do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58122271
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21354/consoleFull) for   PR 2607 at commit [`4784091`](https://github.com/apache/spark/commit/47840911d5f620c0236d0b34520560add3cad833).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19569553
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity}
     import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
     import org.apache.spark.mllib.tree.configuration.Algo._
    -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    --- End diff --
    
    @codedeft I started with a separate model for boosting but @jkbradley (quite correctly IMO) convinced me otherwise. :-)
    
    I agree methods like boosting require support such as early stopping, sequential selection of models, etc. but may be we can handle it as a part of the model configuration. AdaBoost and RF in some ways are more similar than AdaBoost and GBT in their combining operation. It might be better to capture all these nuances in one place. Of course, we can always split them later if we end up writing a lot of custom logic for each algorithm. Thoughts?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58984290
  
    I think the basis of this was that users can have a hard time remembering the order of arguments when there are a large number of arguments.  Parameter classes (like the DT Strategy class) are a good fix; would that work?  The train() methods which take all possible parameters get a bit unwieldy.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58988861
  
    I shortened the parameter list to fit within the constraints.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049170
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,480 @@
    +/*
    + * 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.mllib.tree
    +
    +import org.apache.spark.SparkContext._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, BoostingStrategy}
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{GradientBoostingModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): GradientBoostingModel = {
    +    val strategy = boostingStrategy.strategy
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassification(
    --- End diff --
    
    trainClassification --> trainClassifier


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60508514
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22235/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60534078
  
      [Test build #22255 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22255/consoleFull) for   PR 2607 at commit [`0e81906`](https://github.com/apache/spark/commit/0e81906e5798a5b50edaaafaeee9abb553dd6602).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-57834256
  
    @epahomov  If you or your student are able to take a look at this, I'm sure @manishamde would appreciate it.  This PR will hopefully be generalized to include Classification.  It's nice in that it has infrastructure for multiple losses.  Thank you!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049186
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LeastSquaresError.scala ---
    @@ -0,0 +1,45 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Class for least squares error loss calculation.
    + */
    +object LeastSquaresError extends Loss {
    --- End diff --
    
    Can this be named Squared Error?  ("least" is not really needed.)
    Also, can the doc include a mathematical statement of the form of the error?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049169
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,480 @@
    +/*
    + * 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.mllib.tree
    +
    +import org.apache.spark.SparkContext._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, BoostingStrategy}
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{GradientBoostingModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): GradientBoostingModel = {
    +    val strategy = boostingStrategy.strategy
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +  /**
    --- End diff --
    
    newline needed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58122282
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21354/Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495258
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -46,20 +47,63 @@ private[tree] object BaggedPoint {
        * Convert an input dataset into its BaggedPoint representation,
        * choosing subsample counts for each instance.
        * Each subsample has the same number of instances as the original dataset,
    -   * and is created by subsampling with replacement.
    -   * @param input     Input dataset.
    -   * @param numSubsamples  Number of subsamples of this RDD to take.
    -   * @param seed   Random seed.
    -   * @return  BaggedPoint dataset representation
    +   * and is created by subsampling without replacement.
    +   * @param input Input dataset.
    +   * @param subsample Fraction of the training data used for learning decision tree.
    +   * @param numSubsamples Number of subsamples of this RDD to take.
    +   * @param withReplacement Sampling with/without replacement.
    +   * @param seed Random seed.
    +   * @return BaggedPoint dataset representation.
        */
    -  def convertToBaggedRDD[Datum](
    +  def convertToBaggedRDD[Datum] (
           input: RDD[Datum],
    +      subsample: Double,
    --- End diff --
    
    Rename: subsample --> subsamplingRate


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61203242
  
      [Test build #22582 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22582/consoleFull) for   PR 2607 at commit [`ff2a796`](https://github.com/apache/spark/commit/ff2a796f9feabf11fa4a1b541ef853821bd541e5).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58995827
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21719/consoleFull) for   PR 2607 at commit [`823691b`](https://github.com/apache/spark/commit/823691be8a06515798e0f1a85ee567449a4ef354).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60568456
  
      [Test build #22285 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22285/consoleFull) for   PR 2607 at commit [`eff21fe`](https://github.com/apache/spark/commit/eff21fea01393a44c7876542832e752c26cbcd86).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19639431
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,412 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression and binary classification problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +    timer.start("total")
    +    timer.start("init")
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstTreeModel = new DecisionTree(strategy).train(data)
    +    baseLearners(0) = firstTreeModel
    +    baseLearnerWeights(0) = 1.0
    +    val startingModel = new WeightedEnsembleModel(Array(firstTreeModel), Array(1.0), Regression,
    --- End diff --
    
    (learningRate)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58121120
  
    Here is an interesting design discussion:
    
    For trees and RFs, we convert ```input: RDD[LabeledPoint]``` to ```treeInput: RDD[TreePoint]``` and persist it in memory since the same RDD is re-used during tree/forest building. However, for boosting, we need to construct a new tree every new iteration with a modified RDD where the input labels and weights are modified w.r.t. the original dataset. This leads to the repeated conversion from LabeledPoint to TreePoint every boosting iteration.
    
    Here are a few approaches we can take to :
    (1) Cache ```input: RDD[LabeledPoint]``` and then convert to RDD[TreePoint] during each iteration. We also need to decide whether the ```RDD[TreePoint]``` needs to be cached by default.
    (2) Convert ``input: RDD[LabeledPoint]``` to ```treeInput: RDD[TreePoint]``` just once at the start. However, we will need another method to predict using ```TreePoint``` instances instead of the standard` ```LabeledPoint```. With this strategy, we cache ```TreePoint``` and re-use it every iteration and avoid the binning cost (and also possibly save memory) and avoid storing multiple RDDs in the memory.
    
    I have implemented (1) but I think (2) will be worthwhile to try. Any suggestions?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19069043
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LeastSquaresError.scala ---
    @@ -0,0 +1,45 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Class for least squares error loss calculation.
    + */
    +object LeastSquaresError extends Loss {
    --- End diff --
    
    I am making an attempt to add a mathematical statement. Let me know if we need to be more descriptive. I plan to be more formal in the actual documentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61000703
  
    It's a good point about the sequential nature of boosting models being important when doing approximate predictions (using only some of the weak hypotheses); I could imagine that being useful.  Perhaps the generic WeightedEnsembleModel could be subclassed in order to support that kind of extended functionality in the future.
    
    Distributed models sound useful to me, though I suspect applying a sparsifying step (like running Lasso on the outputs of the many trees to choose a subset of trees) might be faster and almost as accurate in many cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58958404
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21705/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58772417
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21653/consoleFull) for   PR 2607 at commit [`6dd4dd8`](https://github.com/apache/spark/commit/6dd4dd82ac6d42be0edb3f7498b300cc121a8021).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60540637
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22264/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19641825
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,412 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression and binary classification problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +    timer.start("total")
    +    timer.start("init")
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstTreeModel = new DecisionTree(strategy).train(data)
    +    baseLearners(0) = firstTreeModel
    +    baseLearnerWeights(0) = 1.0
    --- End diff --
    
    In the Friedman paper, the first "model" is just the average label (for squared error).  I think it's fine to keep it as is; that way, running for just 1 iteration will behave reasonably.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by codedeft <gi...@git.apache.org>.
Github user codedeft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19569497
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    One more thing, I think that the decision tree itself does persisting of discretized data. So it seems that this could potentially require doubly persisted datasets (one LabeledPoint and the other one TreePoint)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58958272
  
    (Ignore that comment from SparkQA; we're debugging some issues with it right now)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58734566
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21611/Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58066951
  
    @manishamde 
    Multi-class classification: Good point; I agree.  I think this implementation can support binary, but we can do another to support multiclass.  (For multiclass, I think the .OC error-correcting versions might be the best options.)
    
    Weighted weak hypotheses: I am OK if this initial PR does not include weights, but then weights should be prioritized for the next update.
    
    For the WeightedEnsemble, that generalization could be part of this PR or a follow-up.
    
    Once this is ready, I'll be happy to help with testing (e.g., to set checkpointing intervals and general performance).
    
    Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59001167
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21719/consoleFull) for   PR 2607 at commit [`823691b`](https://github.com/apache/spark/commit/823691be8a06515798e0f1a85ee567449a4ef354).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19637468
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala ---
    @@ -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.mllib.tree
    +
    +import org.apache.spark.mllib.linalg.Vectors
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel
    +import org.apache.spark.util.StatCounter
    +
    +import scala.collection.mutable
    +
    +object EnsembleTestHelper {
    +
    +  /**
    +   * Aggregates all values in data, and tests whether the empirical mean and stddev are within
    +   * epsilon of the expected values.
    +   * @param data  Every element of the data should be an i.i.d. sample from some distribution.
    +   */
    +  def testRandomArrays(
    +      data: Array[Array[Double]],
    +      numCols: Int,
    +      expectedMean: Double,
    +      expectedStddev: Double,
    +      epsilon: Double) {
    +    val values = new mutable.ArrayBuffer[Double]()
    +    data.foreach { row =>
    +      assert(row.size == numCols)
    +      values ++= row
    +    }
    +    val stats = new StatCounter(values)
    +    assert(math.abs(stats.mean - expectedMean) < epsilon)
    +    assert(math.abs(stats.stdev - expectedStddev) < epsilon)
    +  }
    +
    +  def validateClassifier(
    +      model: WeightedEnsembleModel,
    +      input: Seq[LabeledPoint],
    +      requiredAccuracy: Double) {
    +    val predictions = input.map(x => model.predict(x.features))
    +    val numOffPredictions = predictions.zip(input).count { case (prediction, expected) =>
    +      prediction != expected.label
    +    }
    +    val accuracy = (input.length - numOffPredictions).toDouble / input.length
    +    assert(accuracy >= requiredAccuracy,
    +      s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.")
    +  }
    +
    +  def validateRegressor(
    +      model: WeightedEnsembleModel,
    +      input: Seq[LabeledPoint],
    +      requiredMSE: Double) {
    +    val predictions = input.map(x => model.predict(x.features))
    +    val squaredError = predictions.zip(input).map { case (prediction, expected) =>
    +      val err = prediction - expected.label
    +      err * err
    +    }.sum
    +    val mse = squaredError / input.length
    +    assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.")
    +  }
    +
    +  def generateOrderedLabeledPoints(numFeatures: Int, numInstances: Int): Array[LabeledPoint] = {
    +    val arr = new Array[LabeledPoint](numInstances)
    +    for (i <- 0 until numInstances) {
    +      val label = if (i < numInstances / 10) {
    +        0.0
    +      } else if (i < numInstances / 2) {
    +        1.0
    +      } else if (i < numInstances * 0.9) {
    +        0.0
    +      } else {
    +        1.0
    +      }
    +      val features = Array.fill[Double](numFeatures)(i.toDouble)
    +      arr(i) = new LabeledPoint(label, Vectors.dense(features))
    +    }
    +    arr
    +  }
    +
    +}
    --- End diff --
    
    newline


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61068006
  
      [Test build #22532 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22532/consoleFull) for   PR 2607 at commit [`e33ab61`](https://github.com/apache/spark/commit/e33ab61bfc6e11de6f8a72368de660b90aea1345).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495261
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.SparkContext._
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Class for least squares error loss calculation.
    + *
    + * The features x and the corresponding label y is predicted using the function F.
    + * For each instance:
    + * Loss: (y - F)**2/2
    + * Negative gradient: y - F
    + */
    +object SquaredError extends Loss {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation for least
    +   * squares error calculation.
    +   * @param model Model of the weak learner
    +   * @param point Instance of the training dataset
    +   * @param learningRate Learning rate parameter for regularization
    +   * @return Loss gradient
    +   */
    +  @DeveloperApi
    --- End diff --
    
    You can put the DeveloperApi annotation on the object, rather than on each method.  (Please change elsewhere too)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19498899
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.mllib.tree.configuration
    +
    +import org.apache.spark.annotation.DeveloperApi
    +
    +/**
    + * :: Experimental ::
    + * Enum to select ensemble combining strategy for base learners
    + */
    +@DeveloperApi
    +object EnsembleCombiningStrategy extends Enumeration {
    --- End diff --
    
    You're right that they are a bit different; I was thinking in terms of thresholding for classification, but it would be important to sum, not average, for regression.  I also revoke what I said about supporting things like median.
    This is making me vote for removing EnsembleCombiningStrategy and only supporting sum.  Do you have a use case for average?  (Sorry for the confusion!)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495228
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm.
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation.
    +   * @param model Model of the weak learner.
    +   * @param point Instance of the training dataset.
    +   * @param learningRate Learning rate parameter for regularization.
    +   * @return Loss gradient.
    +   */
    +  @DeveloperApi
    +  def lossGradient(
    +      model: DecisionTreeModel,
    +      point: LabeledPoint,
    +      learningRate: Double): Double
    +
    +  /**
    +   * Method to calculate error of the base learner for the gradient boosting calculation.
    +   * Note: This method is not used by the gradient boosting algorithm but is useful for debugging
    +   * purposes.
    +   * @param model Model of the weak learner.
    +   * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return
    +   */
    +  @DeveloperApi
    +  def computeError(model: DecisionTreeModel, data: RDD[LabeledPoint]): Double
    --- End diff --
    
    Rename to "compute" or "loss"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60512855
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22241/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049173
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,480 @@
    +/*
    + * 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.mllib.tree
    +
    +import org.apache.spark.SparkContext._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, BoostingStrategy}
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{GradientBoostingModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): GradientBoostingModel = {
    +    val strategy = boostingStrategy.strategy
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]],
    +   *                  [[org.apache.spark.mllib.tree.impurity.Entropy]].
    +   *                 Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]].
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): GradientBoostingModel = {
    +    val lossType = Losses.fromString(loss)
    +    val impurityType = Impurities.fromString(impurity)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      impurityType, maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassification(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): GradientBoostingModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val trees = new Array[DecisionTreeModel](numEstimators + 1)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val subsample = boostingStrategy.subsample
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    var lastCachedData = input
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    trees(0) = firstModel
    +    logDebug("error of tree = " + meanSquaredError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +
    +    var m = 1
    +    while (m <= numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    +      timer.stop(s"building tree $m")
    +      trees(m) = model
    +      logDebug("error of tree = " + meanSquaredError(model, data))
    +      // Update data with pseudo-residuals
    +      data = data.map(point => LabeledPoint(loss.lossGradient(model, point, learningRate),
    +        point.features))
    +      if (m % checkpointingPeriod == 1 && m != 1) {
    +        lastCachedData.unpersist()
    +      }
    +      // Checkpoint
    +      if (m % checkpointingPeriod == 0) {
    +        data = data.persist(StorageLevel.MEMORY_AND_DISK)
    +        lastCachedData = data
    +      }
    +      m += 1
    +    }
    +
    +    timer.stop("total")
    +
    +    logInfo("Internal timing for DecisionTree:")
    +    logInfo(s"$timer")
    +
    +
    +    // 3. Output classifier
    +    new GradientBoostingModel(trees, strategy.algo)
    +
    +  }
    +
    +  /**
    +   * Calculates the mean squared error for regression.
    +   */
    +  private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = {
    --- End diff --
    
    This should probably just use the given loss function.  That way, if I'm trying to minimize say L1 error, this will track exactly that error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61069532
  
      [Test build #22534 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22534/consoleFull) for   PR 2607 at commit [`0183cb9`](https://github.com/apache/spark/commit/0183cb994641292020b1a28e890d5419105ae204).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59672538
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21896/consoleFull) for   PR 2607 at commit [`6a11c02`](https://github.com/apache/spark/commit/6a11c0249268378b3319644f467daefa8807a899).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by codedeft <gi...@git.apache.org>.
Github user codedeft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19570062
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity}
     import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
     import org.apache.spark.mllib.tree.configuration.Algo._
    -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    --- End diff --
    
    @manishamde Sounds good.
    
    Just a side note. Because RF models tend to be much bigger than boosted ensembles, we've encountered situations where the model was *too* big to fit in a single machine memory. RandomForest model is in a way a good model for embarassingly parallel predictions so a model could potentially reside in a distributed fashion.
    
    But we haven't yet decided whether we really want to do this (i.e. are humongous models really useful in practice and do we really expect crazy scenarios of gigantic models surpassing dozens of GBs?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by codedeft <gi...@git.apache.org>.
Github user codedeft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19563689
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity}
     import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
     import org.apache.spark.mllib.tree.configuration.Algo._
    -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    --- End diff --
    
    I personally think that Boosted Model can be a separate one from RandomForestModel. E.g., it's not inconceivable to have boosted models to use RandomForestModel as its base learners.
    
    And if this were a truly generic weighted ensemble model, then it could probably live outside of tree.model namespace, since boosting at least in theory doesn't care whether base learners are trees or not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773560
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21656/consoleFull) for   PR 2607 at commit [`f62bc48`](https://github.com/apache/spark/commit/f62bc48491bd3cbb9dc99eee8490ca949287238d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58991202
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21714/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58772720
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21654/consoleFull) for   PR 2607 at commit [`2fbc9c7`](https://github.com/apache/spark/commit/2fbc9c74885617ffc61c2fa9add1148e28acaf91).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19637406
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val weakHypotheses: Array[DecisionTreeModel],
    +    val weakHypothesisWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numWeakHypotheses > 0, s"WeightedEnsembleModel cannot be created without weakHypotheses" +
    +    s". Number of weakHypotheses = $weakHypotheses")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = weakHypotheses.map(learner => learner.predict(features))
    +    if (numWeakHypotheses == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numWeakHypotheses) {
    +        prediction += weakHypothesisWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    algo match {
    +      case Regression => predictRaw(features)
    +      case Classification => {
    +        // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info.
    +        if (predictRaw(features) > 0 ) 1.0 else 0.0
    +      }
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point.
    +   *
    +   * @param features array representing a single data point
    +   * @return Double prediction from the trained model
    +   */
    +  def predictByAveraging(features: Vector): Double = {
    +    algo match {
    +      case Classification =>
    +        val predictionToCount = new mutable.HashMap[Int, Int]()
    +        weakHypotheses.foreach { learner =>
    +          val prediction = learner.predict(features).toInt
    +          predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1
    +        }
    +        predictionToCount.maxBy(_._2)._1
    +      case Regression =>
    +        weakHypotheses.map(_.predict(features)).sum / weakHypotheses.size
    +    }
    +  }
    +
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  def predict(features: Vector): Double = {
    +    combiningStrategy match {
    +      case Sum => predictBySumming(features)
    +      case Average => predictByAveraging(features)
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown combining parameter: $combiningStrategy.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for the given data set.
    +   *
    +   * @param features RDD representing data points to be predicted
    +   * @return RDD[Double] where each entry contains the corresponding prediction
    +   */
    +  def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x))
    +
    +  /**
    +   * Print full model.
    +   */
    +  override def toString: String = {
    +    val header = algo match {
    +      case Classification =>
    +        s"WeightedEnsembleModel classifier with $numWeakHypotheses trees\n"
    +      case Regression =>
    +        s"WeightedEnsembleModel regressor with $numWeakHypotheses trees\n"
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +    header + weakHypotheses.zipWithIndex.map { case (learner, treeIndex) =>
    --- End diff --
    
    I don't think toString() should print the full model.  toString should be concise, and toDebugString should print the full model.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773406
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21655/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61069535
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22534/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60539296
  
      [Test build #22263 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22263/consoleFull) for   PR 2607 at commit [`781542a`](https://github.com/apache/spark/commit/781542a29ffa480c54a8aff67f5344ac698613fb).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049183
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LeastAbsoluteError.scala ---
    @@ -0,0 +1,45 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Class for least absolute error loss calculation.
    + */
    +object LeastAbsoluteError extends Loss {
    --- End diff --
    
    Can this be named Absolute Error?  ("least" is not really needed.)
    Also, can the doc include a mathematical statement of the form of the error?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60537480
  
      [Test build #22264 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22264/consoleFull) for   PR 2607 at commit [`a32a5ab`](https://github.com/apache/spark/commit/a32a5ab4a86440b6d68999fd86bc18fbe213c855).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496561
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm.
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation.
    +   * @param model Model of the weak learner.
    +   * @param point Instance of the training dataset.
    +   * @param learningRate Learning rate parameter for regularization.
    +   * @return Loss gradient.
    +   */
    +  @DeveloperApi
    +  def lossGradient(
    --- End diff --
    
    Technically negative of the gradient. I can rename it to gradient but it might be confusing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60568462
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22285/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58070240
  
    @jkbradley error-correcting codes will be a good option to support though we should also have a generic one-vs-all classifier. Yes, weight support will definitely be a part of the adaboost PR. Let's discuss the WeightedEnsemble as we get close to completing the PR.
    
    Thanks for helping with the testing. I am currently implementing the TreeRdd caching and subsampling without replacement. After that, we can start testing in parallel along with further code development.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61069736
  
    @jkbradley @codedeft I think I have implemented all the suggestions on the PR except for 1) public API and 2) warning when using non SquaredError loss functions. I will work on them next.
    
    In the latest version, I have simplified the algorithm implementation and removed the need for checkpointing since the dataset for each iteration is calculated using the cached input training dataset and the partial GBT model till that iteration. I think this implementation will avoid a lot of checkpointing/caching overhead and lead to a simpler implementation. Could you please take a look at the logic of ```boost``` method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59680403
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21902/consoleFull) for   PR 2607 at commit [`9366b8f`](https://github.com/apache/spark/commit/9366b8f52a31a10cab999f63872b248bddfeaffe).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58989143
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21714/consoleFull) for   PR 2607 at commit [`5b67102`](https://github.com/apache/spark/commit/5b67102bdd5071365d948f90464bc6bbc67d2089).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61079669
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22536/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60507725
  
      [Test build #22235 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22235/consoleFull) for   PR 2607 at commit [`9bc6e74`](https://github.com/apache/spark/commit/9bc6e74ba546763a19380d647fd2ee3c74493c4b).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495237
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val baseLearners: Array[DecisionTreeModel],
    +    val baseLearnerWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numTrees > 0, s"WeightedEnsembleModel cannot be created without base learners. Number " +
    +    s"of baselearners = $baseLearners")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = baseLearners.map(learner => learner.predict(features))
    +    if (numTrees == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numTrees) {
    +        prediction += baseLearnerWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    val treePredictions = baseLearners.map(learner => learner.predict(features))
    +    val rawPrediction = {
    +      if (numTrees == 1) {
    +        treePredictions(0)
    +      } else {
    +        var prediction = treePredictions(0)
    +        var index = 1
    +        while (index < numTrees) {
    +          prediction += baseLearnerWeights(index) * treePredictions(index)
    +          index += 1
    +        }
    +        prediction
    +      }
    +    }
    +    algo match {
    +      case Regression => predictRaw(features)
    +      case Classification => {
    +        // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info.
    +        if (predictRaw(features) > 0 ) 1.0 else 0.0
    +      }
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point.
    +   *
    +   * @param features array representing a single data point
    +   * @return Double prediction from the trained model
    +   */
    +  def predictByAveraging(features: Vector): Double = {
    +    algo match {
    +      case Classification =>
    +        val predictionToCount = new mutable.HashMap[Int, Int]()
    +        baseLearners.foreach { learner =>
    +          val prediction = learner.predict(features).toInt
    +          predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1
    +        }
    +        predictionToCount.maxBy(_._2)._1
    +      case Regression =>
    +        baseLearners.map(_.predict(features)).sum / baseLearners.size
    +    }
    +  }
    +
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  def predict(features: Vector): Double = {
    +    combiningStrategy match {
    +      case Sum => predictBySumming(features)
    +      case Average => predictByAveraging(features)
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown combining parameter: $combiningStrategy.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for the given data set.
    +   *
    +   * @param features RDD representing data points to be predicted
    +   * @return RDD[Double] where each entry contains the corresponding prediction
    +   */
    +  def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x))
    +
    +  /**
    +   * Print full model.
    +   */
    +  override def toString: String = {
    +    val header = algo match {
    +      case Classification =>
    +        s"WeightedEnsembleModel classifier with $numTrees trees\n"
    +      case Regression =>
    +        s"WeightedEnsembleModel regressor with $numTrees trees\n"
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +    header + baseLearners.zipWithIndex.map { case (learner, treeIndex) =>
    +      s"  Tree $treeIndex:\n" + learner.topNode.subtreeToString(4)
    +    }.fold("")(_ + _)
    +  }
    +
    +  /**
    +   * Print the full model to a string.
    +   */
    +  def toDebugString: String = {
    +    val header = toString + "\n"
    +    header + baseLearners.zipWithIndex.map { case (tree, treeIndex) =>
    +      s"  Tree $treeIndex:\n" + tree.topNode.subtreeToString(4)
    +    }.fold("")(_ + _)
    +  }
    +
    +
    +  // TODO: Remove these helpers methods once class is generalized to support any base learning
    +  // algorithms.
    +
    +  /**
    +   * Get number of trees in forest.
    +   */
    +  def numTrees: Int = baseLearners.size
    --- End diff --
    
    Rename to numWeakHypotheses and keep?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773260
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21654/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60812919
  
    @manishamde  Added comments based on a quick pass looking mainly at the API.  My main concern is the same as in my comment above about the verbosity of (a) the many GradientBoosting.train* methods and (b) BoostingStrategy.  Could you please respond to the comment above about making these more modular for easier construction?  Thanks!  In the meantime, I'll make a more detailed pass over the internals.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61070581
  
      [Test build #22535 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22535/consoleFull) for   PR 2607 at commit [`8476b6b`](https://github.com/apache/spark/commit/8476b6b84a94b78708ec34e9a507184c4431d1a6).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59674719
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21897/consoleFull) for   PR 2607 at commit [`9b2e35e`](https://github.com/apache/spark/commit/9b2e35eca543527dff40c091d8ba8a77e83d4c7a).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-57966546
  
    @jkbradley 
    
    I meant multi-class classification. As you pointed out, binary classification should be similar to the regression case but I am not sure one can handle multi-class classification with one tree. We might have to resort to a one-vs-all strategy. I also agree with you on the naming convention -- log loss or negative binomial log likehood are better names.
    
    Yes, I plan to handle weighted weak hypothesis. In fact, I needed it for something like AdaBoost and had to remove it before submitting this PR. Do you think it makes sense to do it along with this PR or do it in the subsequent AdaBoost PR?
    
    I agree about the WeightedEnsemble model. Let me add it to the TODO list. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61078172
  
      [Test build #22535 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22535/consoleFull) for   PR 2607 at commit [`8476b6b`](https://github.com/apache/spark/commit/8476b6b84a94b78708ec34e9a507184c4431d1a6).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049181
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -73,7 +115,8 @@ private[tree] object BaggedPoint {
         }
       }
     
    -  def convertToBaggedRDDWithoutSampling[Datum](input: RDD[Datum]): RDD[BaggedPoint[Datum]] = {
    +  private[tree] def convertToBaggedRDDWithoutSampling[Datum]
    +      (input: RDD[Datum]): RDD[BaggedPoint[Datum]] = {
    --- End diff --
    
    parenthesis on first line:
    ```
    private[tree] def convertToBaggedRDDWithoutSampling[Datum](
        input: RDD[Datum]): RDD[BaggedPoint[Datum]] = {
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58088284
  
    Sounds good!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60820308
  
    @manishamde  Thanks in advance for the API simplification!
    
    Also, I'm realizing that this code should be correct for SquaredError but might not be quite right for the other losses.  Looking at Friedman's paper, I believe that the weak hypothesis weight needs to be adjusted according to the loss.  That calculation is simple for squared error, but it could get complicated for absolute error and logistic loss (requiring median calculations and general convex optimization, respectively, I'd guess).  I'm OK with leaving those other losses as long as they are marked with warnings.  I believe the code will still do something reasonable, although not quite ideal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495241
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala ---
    @@ -0,0 +1,208 @@
    +/*
    + * 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.mllib.tree
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy}
    +import org.apache.spark.mllib.tree.impurity.{Variance, Gini}
    +import org.apache.spark.mllib.tree.loss.{SquaredError, LogLoss}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +
    +import org.apache.spark.mllib.util.LocalSparkContext
    +
    +/**
    + * Test suite for [[GradientBoosting]].
    + */
    +class GradientBoostingSuite extends FunSuite with LocalSparkContext {
    +
    +  test("Binary classification with continuous features:" +
    +    " comparing DecisionTree vs. GradientBoosting (numEstimators = 1)") {
    +
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numEstimators = 1
    +
    +    val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +    val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val dt = DecisionTree.train(remappedInput, treeStrategy)
    +
    +    val boostingStrategy = new BoostingStrategy(algo = Classification,
    +      numEstimators = numEstimators, loss = LogLoss, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy)
    +    assert(gbt.baseLearners.size === 1)
    +    val gbtTree = gbt.baseLearners(0)
    +
    +
    +    EnsembleTestHelper.validateClassifier(gbt, arr, 0.9)
    +
    +    // Make sure trees are the same.
    +    assert(gbtTree.toString == dt.toString)
    +  }
    +
    +  test("Binary classification with continuous features:" +
    +    " comparing DecisionTree vs. GradientBoosting (numEstimators = 10)") {
    +
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numEstimators = 10
    +
    +    val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +    val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val dt = DecisionTree.train(remappedInput, treeStrategy)
    +
    +    val boostingStrategy = new BoostingStrategy(algo = Classification,
    +      numEstimators = numEstimators, loss = LogLoss, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy)
    +    assert(gbt.baseLearners.size === 10)
    +    val gbtTree = gbt.baseLearners(0)
    +
    +
    +    EnsembleTestHelper.validateClassifier(gbt, arr, 0.9)
    +
    +    // Make sure trees are the same.
    +    assert(gbtTree.toString == dt.toString)
    +  }
    +
    +  test("Binary classification with continuous features:" +
    +    " Stochastic GradientBoosting (numEstimators = 10, learning rate = 0.9, subsample = 0.75)") {
    +
    +    val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numEstimators = 10
    +
    +    val boostingStrategy = new BoostingStrategy(algo = Classification,
    +      numEstimators = numEstimators, loss = LogLoss, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo,
    +      subsample = 0.75)
    +
    +    val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy)
    +    assert(gbt.baseLearners.size === 10)
    +
    +    EnsembleTestHelper.validateClassifier(gbt, arr, 0.9)
    +
    +  }
    +
    +
    --- End diff --
    
    extra spaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049198
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/GradientBoostingModel.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.rdd.RDD
    +
    +
    +class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable {
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  def predict(features: Vector): Double = {
    +    trees.map(tree => tree.predict(features)).sum
    --- End diff --
    
    Could this be changed to match the predict() method for RandomForest?  Currently, this is more of a predictRaw() method.  (Also, should this be mean, not sum?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59697998
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21908/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496266
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val baseLearners: Array[DecisionTreeModel],
    +    val baseLearnerWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numTrees > 0, s"WeightedEnsembleModel cannot be created without base learners. Number " +
    +    s"of baselearners = $baseLearners")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = baseLearners.map(learner => learner.predict(features))
    +    if (numTrees == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numTrees) {
    +        prediction += baseLearnerWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    val treePredictions = baseLearners.map(learner => learner.predict(features))
    +    val rawPrediction = {
    +      if (numTrees == 1) {
    +        treePredictions(0)
    +      } else {
    +        var prediction = treePredictions(0)
    +        var index = 1
    +        while (index < numTrees) {
    +          prediction += baseLearnerWeights(index) * treePredictions(index)
    +          index += 1
    +        }
    +        prediction
    +      }
    +    }
    +    algo match {
    +      case Regression => predictRaw(features)
    +      case Classification => {
    +        // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info.
    +        if (predictRaw(features) > 0 ) 1.0 else 0.0
    +      }
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point.
    +   *
    +   * @param features array representing a single data point
    +   * @return Double prediction from the trained model
    +   */
    +  def predictByAveraging(features: Vector): Double = {
    +    algo match {
    +      case Classification =>
    +        val predictionToCount = new mutable.HashMap[Int, Int]()
    +        baseLearners.foreach { learner =>
    +          val prediction = learner.predict(features).toInt
    +          predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1
    +        }
    +        predictionToCount.maxBy(_._2)._1
    +      case Regression =>
    +        baseLearners.map(_.predict(features)).sum / baseLearners.size
    +    }
    +  }
    +
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  def predict(features: Vector): Double = {
    +    combiningStrategy match {
    +      case Sum => predictBySumming(features)
    +      case Average => predictByAveraging(features)
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown combining parameter: $combiningStrategy.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for the given data set.
    +   *
    +   * @param features RDD representing data points to be predicted
    +   * @return RDD[Double] where each entry contains the corresponding prediction
    +   */
    +  def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x))
    +
    +  /**
    +   * Print full model.
    +   */
    +  override def toString: String = {
    +    val header = algo match {
    +      case Classification =>
    +        s"WeightedEnsembleModel classifier with $numTrees trees\n"
    +      case Regression =>
    +        s"WeightedEnsembleModel regressor with $numTrees trees\n"
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +    header + baseLearners.zipWithIndex.map { case (learner, treeIndex) =>
    +      s"  Tree $treeIndex:\n" + learner.topNode.subtreeToString(4)
    +    }.fold("")(_ + _)
    +  }
    +
    +  /**
    +   * Print the full model to a string.
    +   */
    +  def toDebugString: String = {
    +    val header = toString + "\n"
    +    header + baseLearners.zipWithIndex.map { case (tree, treeIndex) =>
    +      s"  Tree $treeIndex:\n" + tree.topNode.subtreeToString(4)
    +    }.fold("")(_ + _)
    +  }
    +
    +
    +  // TODO: Remove these helpers methods once class is generalized to support any base learning
    +  // algorithms.
    +
    +  /**
    +   * Get number of trees in forest.
    +   */
    +  def numTrees: Int = baseLearners.size
    --- End diff --
    
    Agree.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773003
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21656/consoleFull) for   PR 2607 at commit [`f62bc48`](https://github.com/apache/spark/commit/f62bc48491bd3cbb9dc99eee8490ca949287238d).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-57554926
  
    @manishamde  Thanks for the WIP PR!
    
    About classification, what points need to be discussed?  Why is it more difficult to figure out than regression?  (Also, I personally am not a big fan of the name "deviance" even though it is used in sklearn and in Friedman's paper.  I prefer more descriptive names like LogLoss.)
    
    Also, will this be generalized to support weighted weak hypotheses, common in most boosting algorithms?
    
    For the final Model produced, should we use the same class for both random forests and gradient boosting?  It could be a TreeEnsemble model (to be generalized later to a WeightedEnsemble model).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58734564
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21611/consoleFull) for   PR 2607 at commit [`62cc000`](https://github.com/apache/spark/commit/62cc000cb39859a21d716fcf129c0390ce842173).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496447
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val baseLearners: Array[DecisionTreeModel],
    --- End diff --
    
    Sure. Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60690690
  
    @jkbradley I fixed the merge conflicts. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58956391
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21703/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049189
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Class for least squares error loss calculation.
    + */
    +object LogLoss extends Loss {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation for binary
    +   * classification
    +   * @param model Model of the weak learner
    +   * @param point Instance of the training dataset
    +   * @param learningRate Learning rate parameter for regularization
    +   * @return Loss gradient
    +   */
    +  @DeveloperApi
    +  override def lossGradient(
    +                             model: DecisionTreeModel,
    +                             point: LabeledPoint,
    +                             learningRate: Double): Double = {
    +    val prediction = model.predict(point.features)
    --- End diff --
    
    predict() will return the class.  We'll need a predictRaw() method.  Perhaps that can be implemented in a separate, small PR first.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19498319
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm.
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation.
    +   * @param model Model of the weak learner.
    +   * @param point Instance of the training dataset.
    +   * @param learningRate Learning rate parameter for regularization.
    +   * @return Loss gradient.
    --- End diff --
    
    Since this is called gradient, could it please return the gradient instead of the negated gradient?  (And boosting can be updated accordingly.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19645493
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,412 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression and binary classification problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +    timer.start("total")
    +    timer.start("init")
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstTreeModel = new DecisionTree(strategy).train(data)
    +    baseLearners(0) = firstTreeModel
    +    baseLearnerWeights(0) = 1.0
    --- End diff --
    
    Yup.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773733
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21657/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61222744
  
    True, it's a good point about LR.  OK, let's keep them with caveats, but hopefully run some tests to make sure they seem to be working.  I'll make a pass tomorrow morning; thanks for the updates!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61079666
  
      [Test build #22536 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22536/consoleFull) for   PR 2607 at commit [`b4c1318`](https://github.com/apache/spark/commit/b4c13188de93f4e337bb82405de5969ae22cbdb9).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59674398
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21896/consoleFull) for   PR 2607 at commit [`6a11c02`](https://github.com/apache/spark/commit/6a11c0249268378b3319644f467daefa8807a899).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60696746
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22313/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773403
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21655/consoleFull) for   PR 2607 at commit [`bdca43a`](https://github.com/apache/spark/commit/bdca43a4679f194c22209733852515cfa09bf407).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58396977
  
    @jkbradley Cool. I will proceed forward with the remaining tasks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19497886
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    --- End diff --
    
    extra newline


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58288947
  
    @jkbradley Cool. I am sure we will see a definitely performance gain once we implement support for (2) once we have a standard API.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61196088
  
      [Test build #22582 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22582/consoleFull) for   PR 2607 at commit [`ff2a796`](https://github.com/apache/spark/commit/ff2a796f9feabf11fa4a1b541ef853821bd541e5).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58772989
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21653/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60531596
  
      [Test build #22255 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22255/consoleFull) for   PR 2607 at commit [`0e81906`](https://github.com/apache/spark/commit/0e81906e5798a5b50edaaafaeee9abb553dd6602).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61069431
  
      [Test build #22534 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22534/consoleFull) for   PR 2607 at commit [`0183cb9`](https://github.com/apache/spark/commit/0183cb994641292020b1a28e890d5419105ae204).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60540632
  
      [Test build #22264 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22264/consoleFull) for   PR 2607 at commit [`a32a5ab`](https://github.com/apache/spark/commit/a32a5ab4a86440b6d68999fd86bc18fbe213c855).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59676890
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21897/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59690664
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21908/consoleFull) for   PR 2607 at commit [`2ae97b7`](https://github.com/apache/spark/commit/2ae97b74ccc0e7fc3f34d435264768a1403a7a0c).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60696743
  
      [Test build #22313 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22313/consoleFull) for   PR 2607 at commit [`49ba107`](https://github.com/apache/spark/commit/49ba107065e0c53beee316e7108f7900a49b47e1).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60824862
  
    @jkbradley Should we even support classification then?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61026909
  
    Studying the trade-offs sounds great.  I think it's OK if checkpointing is added later as an option.  Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19564695
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    Based on @manishamde 's PR description, I think the plan is to do this optimization later.  Keeping it a separate PR is helpful for reducing conflict with your node ID caching PR [https://github.com/apache/spark/pull/2868].  I feel like it is easier to break things into smaller PRs.  Also, since this type of optimization will likely be useful for other meta-algorithms, it will be good to think about a standard interface for getting a learning algorithm's internal data representation (and the related prediction methods which take that internal representation).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495234
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val baseLearners: Array[DecisionTreeModel],
    +    val baseLearnerWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numTrees > 0, s"WeightedEnsembleModel cannot be created without base learners. Number " +
    +    s"of baselearners = $baseLearners")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = baseLearners.map(learner => learner.predict(features))
    +    if (numTrees == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numTrees) {
    +        prediction += baseLearnerWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    val treePredictions = baseLearners.map(learner => learner.predict(features))
    +    val rawPrediction = {
    --- End diff --
    
    Remember to remove (since you moved it to predictRaw)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61225506
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22596/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60561667
  
      [Test build #22285 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22285/consoleFull) for   PR 2607 at commit [`eff21fe`](https://github.com/apache/spark/commit/eff21fea01393a44c7876542832e752c26cbcd86).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58118535
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21353/consoleFull) for   PR 2607 at commit [`78ed452`](https://github.com/apache/spark/commit/78ed452a522a651d54e4cb5d5381a2ee5c559270).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61172803
  
    @manishamde  The logic looks better (especially since you caught the learningRate bug!).  After the API update (train*, BoostingStrategy, and making AbsoluteError and LogLoss private), I think this will be ready.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59674403
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21896/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61203248
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22582/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60824501
  
    @manishamde Thinking more about the losses, I'm really not sure if absolute error and logistic loss will behave reasonably.  Could we make those losses private[tree] and mark them with TODO?  That way, your code is not thrown out, but we won't expose unsafe options to users.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19564926
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity}
     import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
     import org.apache.spark.mllib.tree.configuration.Algo._
    -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    --- End diff --
    
    These generalizations will rely on the new ML API (for which there will be a PR any day now); it makes sense to keep it in the tree namespace since there is not generic Estimator concept currently.  But once we can, I agree it will be important to generalize meta-algorithms.
    
    With respect to the models, I don't see how the model concepts are different.  The learning algorithms are different, but that will not prevent a meta-algorithm to use another meta-algorithm as a weak learner (once the new API is available).  (I think it's good to separate the concepts of Estimator (learning algorithm) and Transformer (learned model) here.)  What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049187
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +
    +/**
    + * Class for least squares error loss calculation.
    + */
    +object LogLoss extends Loss {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation for binary
    +   * classification
    +   * @param model Model of the weak learner
    +   * @param point Instance of the training dataset
    +   * @param learningRate Learning rate parameter for regularization
    +   * @return Loss gradient
    +   */
    +  @DeveloperApi
    +  override def lossGradient(
    +                             model: DecisionTreeModel,
    --- End diff --
    
    spacing (4 spaces indentation)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61077355
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22533/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049176
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -47,20 +48,61 @@ private[tree] object BaggedPoint {
        * Convert an input dataset into its BaggedPoint representation,
        * choosing subsample counts for each instance.
        * Each subsample has the same number of instances as the original dataset,
    -   * and is created by subsampling with replacement.
    +   * and is created by subsampling without replacement.
        * @param input     Input dataset.
    +   * @param subsample Fraction of the training data used for learning decision tree.
        * @param numSubsamples  Number of subsamples of this RDD to take.
    -   * @param seed   Random seed.
    --- End diff --
    
    I think we should keep this parameter.  That will allow reproducible results.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58113485
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21351/consoleFull) for   PR 2607 at commit [`3973dd1`](https://github.com/apache/spark/commit/3973dd1a41f13e3fb58d6c20bfed827403f7d270).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773142
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21657/consoleFull) for   PR 2607 at commit [`8e10c63`](https://github.com/apache/spark/commit/8e10c6364cf9ac0fffbfc63e1234a8e44c77a640).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61155798
  
    @jkbradley I agree with protection against driver failure for long sequential operations. However, in this case we will just be checkpointing partial models rather than the intermediate datasets similar to other iterative algorithms such as LR. Look forward to your feedback on the new logic.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60508513
  
      [Test build #22235 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22235/consoleFull) for   PR 2607 at commit [`9bc6e74`](https://github.com/apache/spark/commit/9bc6e74ba546763a19380d647fd2ee3c74493c4b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], strategy: BoostingStrategy)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by codedeft <gi...@git.apache.org>.
Github user codedeft commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19567808
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    Makes sense. So I suppose you want to provide the functionality first and then optimize later ;).
    
    I'm not sure though about whether this is going to result in re-reading from the disk the input at every iteration. Maybe I'm wrong. But a simple change could be simply persisting features all the time, and re-persisting newly calculated labels periodically.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61220989
  
      [Test build #22596 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22596/consoleFull) for   PR 2607 at commit [`991c7b5`](https://github.com/apache/spark/commit/991c7b58f4648693e7b01ef756d032cc51980eec).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19499795
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19508317
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    --- End diff --
    
    Could this please be renamed to checkpointInterval everywhere?  I just noticed this name is used throughout other parts of Spark, so we can follow that precedent.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59680411
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21902/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58847116
  
    @manishamde It looks like the failure was in compilation:
    ```
    [error] /home/jenkins/workspace/SparkPullRequestBuilder/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala:430: method convertToBaggedRDDWithoutSampling in object BaggedPoint cannot be accessed in object org.apache.spark.mllib.tree.impl.BaggedPoint
    [error]     val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    [error]                                   ^
    [warn] two warnings found
    [error] /home/jenkins/workspace/SparkPullRequestBuilder/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala:723: method convertToBaggedRDDWithoutSampling in object BaggedPoint cannot be accessed in object org.apache.spark.mllib.tree.impl.BaggedPoint
    [error]     val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    [error]                                   ^
    [error] /home/jenkins/workspace/SparkPullRequestBuilder/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala:766: method convertToBaggedRDDWithoutSampling in object BaggedPoint cannot be accessed in object org.apache.spark.mllib.tree.impl.BaggedPoint
    [error]     val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    [error]                                   ^
    [error] /home/jenkins/workspace/SparkPullRequestBuilder/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala:42: method convertToBaggedRDDWithoutSampling in object BaggedPoint cannot be accessed in object org.apache.spark.mllib.tree.impl.BaggedPoint
    [error]     val baggedRDD = BaggedPoint.convertToBaggedRDDWithoutSampling(rdd)
    [error]                                 ^
    [error] /home/jenkins/workspace/SparkPullRequestBuilder/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala:56: not enough arguments for method convertToBaggedRDD: (input: org.apache.spark.rdd.RDD[Datum], subsample: Double, numSubsamples: Int, withReplacement: Boolean)org.apache.spark.rdd.RDD[org.apache.spark.mllib.tree.impl.BaggedPoint[Datum]].
    [error] Unspecified value parameter withReplacement.
    [error]       val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, numSubsamples, seed = seed)
    [error]                                                     ^
    [error] 5 errors found
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19497878
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    --- End diff --
    
    GradientBoostingModel --> WeightedEnsembleModel


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60837931
  
    I think it's OK to leave classification support but make a note in the doc for SquaredError that it is meant for Regression.  What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58119278
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21354/consoleFull) for   PR 2607 at commit [`4784091`](https://github.com/apache/spark/commit/47840911d5f620c0236d0b34520560add3cad833).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496113
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala ---
    @@ -70,7 +71,8 @@ class Strategy (
         val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](),
         val minInstancesPerNode: Int = 1,
         val minInfoGain: Double = 0.0,
    -    val maxMemoryInMB: Int = 256) extends Serializable {
    +    val maxMemoryInMB: Int = 256,
    +    val subsample: Double = 1) extends Serializable {
    --- End diff --
    
    Agree. Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60539297
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22263/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58994175
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21716/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61225502
  
      [Test build #22596 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22596/consoleFull) for   PR 2607 at commit [`991c7b5`](https://github.com/apache/spark/commit/991c7b58f4648693e7b01ef756d032cc51980eec).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61067173
  
      [Test build #22531 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22531/consoleFull) for   PR 2607 at commit [`035a2ed`](https://github.com/apache/spark/commit/035a2ed6bb09910a3e8a6593b3276b742cf7b7d2).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58120548
  
    I have added stochastic gradient boosting by adding code for subsampling without replacement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60534080
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22255/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58733404
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21611/consoleFull) for   PR 2607 at commit [`62cc000`](https://github.com/apache/spark/commit/62cc000cb39859a21d716fcf129c0390ce842173).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19495254
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala ---
    @@ -70,7 +71,8 @@ class Strategy (
         val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](),
         val minInstancesPerNode: Int = 1,
         val minInfoGain: Double = 0.0,
    -    val maxMemoryInMB: Int = 256) extends Serializable {
    +    val maxMemoryInMB: Int = 256,
    +    val subsample: Double = 1) extends Serializable {
    --- End diff --
    
    Rename: subsample --> subsamplingRate


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60824790
  
    @jkbradley I agree. This needs more testing since it's a non-standard option.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60686705
  
    @manishamde   I'll make a pass now; thanks for the updates!  A patch (SPARK-4022) was just merged which causes a few small conflicts.  Could you please fix those?  Then I'll do some tests.  Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049168
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,480 @@
    +/*
    + * 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.mllib.tree
    +
    +import org.apache.spark.SparkContext._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, BoostingStrategy}
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{GradientBoostingModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): GradientBoostingModel = {
    +    val strategy = boostingStrategy.strategy
    --- End diff --
    
    "strategy" is not used


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60814290
  
    @jkbradley Your API suggestions sound reasonable. Let me work on simplifying the API. I had originally started with something similar to what you suggested so I will revert to that. I will let you know once the API has been updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19049196
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/GradientBoostingModel.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.rdd.RDD
    +
    +
    +class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable {
    --- End diff --
    
    Mark as @Experimental?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58113486
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21351/Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61345885
  
    @mengxr Could we get this merged? :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19498943
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm.
    + */
    +trait Loss extends Serializable {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation.
    +   * @param model Model of the weak learner.
    +   * @param point Instance of the training dataset.
    +   * @param learningRate Learning rate parameter for regularization.
    +   * @return Loss gradient.
    --- End diff --
    
    (just saw your other note; I vote for renaming to gradient, and making it actually return the gradient)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60863725
  
    By the way, checkpointing is not quite the right term; currently, the code persists but does not checkpoint the RDDs.  I hope that the logic which @codedeft implemented in another PR [https://github.com/apache/spark/pull/2868] can be abstracted away to be reused for boosting.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61147592
  
    Thanks for the updates; I'll take a look.  I think that it will be very important to include checkpointing, but I am OK with adding it later on.  (Since boosting is sequential, I could imagine it running for much longer than bagging/forest algorithms, so protecting against driver failure will be important.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61024859
  
    @jkbradley I originally used checkpointing instead of simply caching in memory. There are trade-offs going with one versus the other. I will study what @codedeft implemented in PR [#2868](https://github.com/apache/spark/pull/2868) and see what we can re-use here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19510078
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    --- End diff --
    
    Sure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59676884
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21897/consoleFull) for   PR 2607 at commit [`9b2e35e`](https://github.com/apache/spark/commit/9b2e35eca543527dff40c091d8ba8a77e83d4c7a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19570610
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity}
     import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
     import org.apache.spark.mllib.tree.configuration.Algo._
    -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    --- End diff --
    
    @codedeft Agree about the distributed storage though I never bothered to check the size of big deep trees in memory! :-) In fact, such a storage might be a good option for [Partial Forest implementation](https://issues.apache.org/jira/browse/SPARK-1548).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773562
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21656/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19496069
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.mllib.tree.loss
    +
    +import org.apache.spark.SparkContext._
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.DecisionTreeModel
    +import org.apache.spark.rdd.RDD
    +
    +/**
    + * Class for least squares error loss calculation.
    + *
    + * The features x and the corresponding label y is predicted using the function F.
    + * For each instance:
    + * Loss: (y - F)**2/2
    + * Negative gradient: y - F
    + */
    +object SquaredError extends Loss {
    +
    +  /**
    +   * Method to calculate the loss gradients for the gradient boosting calculation for least
    +   * squares error calculation.
    +   * @param model Model of the weak learner
    +   * @param point Instance of the training dataset
    +   * @param learningRate Learning rate parameter for regularization
    +   * @return Loss gradient
    +   */
    +  @DeveloperApi
    --- End diff --
    
    Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19569087
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    @codedeft Thanks for your comment. Your observation is correct. Conversion to the internal discretized/binned storage format will definitely lead to a faster implementation and lower memory consumption on the cluster. As @jkbradley mentioned, we decided to work on it after the generic MLlib work has been completed. We can then use methods such as ```trainUsingInternalFormat``` and ```predictUsingInternalFormat``` if the underlying algo (in this case DecisionTree) supports it.
    
    We won't be re-reading from disk at every iteration but caching the training data at the first iteration and checkpointing/persisting every few iterations to avoid long lineage chains. Will comment on the checkpointing further in the other thread.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19638832
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val weakHypotheses: Array[DecisionTreeModel],
    +    val weakHypothesisWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numWeakHypotheses > 0, s"WeightedEnsembleModel cannot be created without weakHypotheses" +
    +    s". Number of weakHypotheses = $weakHypotheses")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = weakHypotheses.map(learner => learner.predict(features))
    +    if (numWeakHypotheses == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numWeakHypotheses) {
    +        prediction += weakHypothesisWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    algo match {
    +      case Regression => predictRaw(features)
    +      case Classification => {
    +        // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info.
    +        if (predictRaw(features) > 0 ) 1.0 else 0.0
    +      }
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point.
    +   *
    +   * @param features array representing a single data point
    +   * @return Double prediction from the trained model
    +   */
    +  def predictByAveraging(features: Vector): Double = {
    --- End diff --
    
    will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19639409
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,412 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression and binary classification problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int,
    +        Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsamplingRate  Fraction of the training data used for learning the decision tree.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsamplingRate: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsamplingRate)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return WeightedEnsembleModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +    timer.start("total")
    +    timer.start("init")
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstTreeModel = new DecisionTree(strategy).train(data)
    +    baseLearners(0) = firstTreeModel
    +    baseLearnerWeights(0) = 1.0
    --- End diff --
    
    This should be learningRate too, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19637449
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._
    +import org.apache.spark.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +@Experimental
    +class WeightedEnsembleModel(
    +    val weakHypotheses: Array[DecisionTreeModel],
    +    val weakHypothesisWeights: Array[Double],
    +    val algo: Algo,
    +    val combiningStrategy: EnsembleCombiningStrategy) extends Serializable {
    +
    +  require(numWeakHypotheses > 0, s"WeightedEnsembleModel cannot be created without weakHypotheses" +
    +    s". Number of weakHypotheses = $weakHypotheses")
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictRaw(features: Vector): Double = {
    +    val treePredictions = weakHypotheses.map(learner => learner.predict(features))
    +    if (numWeakHypotheses == 1){
    +      treePredictions(0)
    +    } else {
    +      var prediction = treePredictions(0)
    +      var index = 1
    +      while (index < numWeakHypotheses) {
    +        prediction += weakHypothesisWeights(index) * treePredictions(index)
    +        index += 1
    +      }
    +      prediction
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  private def predictBySumming(features: Vector): Double = {
    +    algo match {
    +      case Regression => predictRaw(features)
    +      case Classification => {
    +        // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info.
    +        if (predictRaw(features) > 0 ) 1.0 else 0.0
    +      }
    +      case _ => throw new IllegalArgumentException(
    +        s"WeightedEnsembleModel given unknown algo parameter: $algo.")
    +    }
    +  }
    +
    +  /**
    +   * Predict values for a single data point.
    +   *
    +   * @param features array representing a single data point
    +   * @return Double prediction from the trained model
    +   */
    +  def predictByAveraging(features: Vector): Double = {
    --- End diff --
    
    private


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-60690948
  
      [Test build #22313 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22313/consoleFull) for   PR 2607 at commit [`49ba107`](https://github.com/apache/spark/commit/49ba107065e0c53beee316e7108f7900a49b47e1).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19499284
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.mllib.tree.configuration
    +
    +import org.apache.spark.annotation.DeveloperApi
    +
    +/**
    + * :: Experimental ::
    + * Enum to select ensemble combining strategy for base learners
    + */
    +@DeveloperApi
    +object EnsembleCombiningStrategy extends Enumeration {
    --- End diff --
    
    I think we use average or majority for the random forest ensemble calculations. I moved the RF code to also return a WeightedEnsembleModel.
    
    I am okay with removing EnsembleCombiningStrategy class but we still need to support both sum and averaging operations for combining ensemble predictions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59001175
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21719/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58113366
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21351/consoleFull) for   PR 2607 at commit [`3973dd1`](https://github.com/apache/spark/commit/3973dd1a41f13e3fb58d6c20bfed827403f7d270).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-59678193
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21899/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19497871
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    --- End diff --
    
    "regression" --> "binary classification and regression"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19499682
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.mllib.tree.configuration
    +
    +import org.apache.spark.annotation.DeveloperApi
    +
    +/**
    + * :: Experimental ::
    + * Enum to select ensemble combining strategy for base learners
    + */
    +@DeveloperApi
    +object EnsembleCombiningStrategy extends Enumeration {
    --- End diff --
    
    OK, I agree; keeping it sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61075782
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22532/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by manishamde <gi...@git.apache.org>.
Github user manishamde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19066105
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/GradientBoostingModel.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.mllib.tree.model
    +
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.rdd.RDD
    +
    +
    +class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable {
    +  /**
    +   * Predict values for a single data point using the model trained.
    +   *
    +   * @param features array representing a single data point
    +   * @return predicted category from the trained model
    +   */
    +  def predict(features: Vector): Double = {
    +    trees.map(tree => tree.predict(features)).sum
    --- End diff --
    
    Sure. 
    
    (weighted) sum / numTrees for RF and AdaBoost and sum (normalized by learning rate) for GBT. Correct?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] [WIP] SPARK-1547: Adding Gradient Boos...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-58773977
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21658/consoleFull) for   PR 2607 at commit [`3b8ffc0`](https://github.com/apache/spark/commit/3b8ffc00e9854ca323f0c8772784bf1337eec562).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GradientBoosting (`
      * `case class BoostingStrategy(`
      * `trait Loss extends Serializable `
      * `class GradientBoostingModel(trees: Array[DecisionTreeModel], algo: Algo) extends Serializable `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Adding Gradient Boosting t...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2607#discussion_r19572600
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala ---
    @@ -0,0 +1,433 @@
    +/*
    + * 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.mllib.tree
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.tree.configuration.BoostingStrategy
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.tree.impl.TimeTracker
    +import org.apache.spark.mllib.tree.loss.Losses
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel}
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum
    +
    +/**
    + * :: Experimental ::
    + * A class that implements gradient boosting for regression problems.
    + * @param boostingStrategy Parameters for the gradient boosting algorithm
    + */
    +@Experimental
    +class GradientBoosting (
    +    private val boostingStrategy: BoostingStrategy) extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    algo match {
    +      case Regression => GradientBoosting.boost(input, boostingStrategy)
    +      case Classification =>
    +        val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features))
    +        GradientBoosting.boost(remappedInput, boostingStrategy)
    +      case _ =>
    +        throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.")
    +    }
    +  }
    +
    +}
    +
    +
    +object GradientBoosting extends Logging {
    +
    +  /**
    +   * Method to train a gradient boosting model.
    +   *
    +   * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *       is recommended to clearly specify regression.
    +   *       Using [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *       is recommended to clearly specify regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def train(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int]): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, numClassesForClassification,
    +      categoricalFeaturesInfo = categoricalFeaturesInfo)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainRegressor]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod, 2, categoricalFeaturesInfo =
    +        categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting#trainClassifier]]
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @param checkpointPeriod Checkpointing the dataset in memory to avoid long lineage chains.
    +   * @param numClassesForClassification Number of classes for classification.
    +   *                                    (Ignored for regression.)
    +   *                                    Default value is 2 (binary classification).
    +   * @param categoricalFeaturesInfo A map storing information about the categorical variables and
    +   *                                the number of discrete values they take. For example,
    +   *                                an entry (n -> k) implies the feature n is categorical with k
    +   *                                categories 0, 1, 2, ... , k-1. It's important to note that
    +   *                                features are zero-indexed.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double,
    +      checkpointPeriod: Int,
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer])
    +      : WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample, checkpointPeriod,
    +      numClassesForClassification, categoricalFeaturesInfo =
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting binary classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param numEstimators Number of estimators used in boosting stages. In other words,
    +   *                      number of boosting iterations performed.
    +   * @param loss Loss function used for minimization during gradient boosting.
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   * @param learningRate Learning rate for shrinking the contribution of each estimator. The
    +   *                     learning rate should be between in the interval (0, 1]
    +   * @param subsample  Fraction of the training data used for learning the decision tree.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numEstimators: Int,
    +      loss: String,
    +      maxDepth: Int,
    +      learningRate: Double,
    +      subsample: Double): WeightedEnsembleModel = {
    +    val lossType = Losses.fromString(loss)
    +    val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType,
    +      maxDepth, learningRate, subsample)
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting regression model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Method to train a gradient boosting classification model.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              For classification, labels should take values {0, 1, ..., numClasses-1}.
    +   *              For regression, labels are real numbers.
    +   * @param boostingStrategy Configuration options for the boosting algorithm.
    +   * @return GradientBoostingModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +    val algo = boostingStrategy.algo
    +    require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.")
    +    new GradientBoosting(boostingStrategy).train(input)
    +  }
    +
    +  /**
    +   * Internal method for performing regression using trees as base learners.
    +   * @param input training dataset
    +   * @param boostingStrategy boosting parameters
    +   * @return
    +   */
    +  private def boost(
    +      input: RDD[LabeledPoint],
    +      boostingStrategy: BoostingStrategy): WeightedEnsembleModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +    timer.start("init")
    +
    +
    +    // Initialize gradient boosting parameters
    +    val numEstimators = boostingStrategy.numEstimators
    +    val baseLearners = new Array[DecisionTreeModel](numEstimators)
    +    val baseLearnerWeights = new Array[Double](numEstimators)
    +    val loss = boostingStrategy.loss
    +    val learningRate = boostingStrategy.learningRate
    +    val checkpointingPeriod = boostingStrategy.checkpointPeriod
    +    val strategy = boostingStrategy.strategy
    +
    +    // Cache input
    +    input.persist(StorageLevel.MEMORY_AND_DISK)
    +    // Dataset reference for keeping track of last cached dataset in memory.
    +    var lastCachedData = input
    +    // Dataset reference for noting dataset marked for unpersisting.
    +    var unpersistData = lastCachedData
    +
    +    timer.stop("init")
    +
    +    logDebug("##########")
    +    logDebug("Building tree 0")
    +    logDebug("##########")
    +    var data = input
    +
    +    // 1. Initialize tree
    +    timer.start("building tree 0")
    +    val firstModel = new DecisionTree(strategy).train(data)
    +    timer.stop("building tree 0")
    +    baseLearners(0) = firstModel
    +    baseLearnerWeights(0) = 1.0
    +    logDebug("error of tree = " + loss.computeError(firstModel, data))
    +
    +    // psuedo-residual for second iteration
    +    data = data.map(point => LabeledPoint(loss.lossGradient(firstModel, point,
    +      learningRate), point.features))
    +
    +    var m = 1
    +    while (m < numEstimators) {
    +      timer.start(s"building tree $m")
    +      logDebug("###################################################")
    +      logDebug("Gradient boosting tree iteration " + m)
    +      logDebug("###################################################")
    +      val model = new DecisionTree(strategy).train(data)
    --- End diff --
    
    Eventually, I envision:
    (1) GradientBoosting gets 1 copy of the data from the weak learner (RDD[TreePoint] for DecisionTree) and persists it.
    (2) DecisionTree persists the NodeIdCache (possibly storing 2 copies of the cache).  GradientBoosting tells DecisionTree not to serialize anything.
    (3) GradientBoosting persists the label (as @codedeft suggested) only, and periodically serializes it.
    
    For now, I saw we either keep it as is and add a warning, or spend a little time refactoring to just persist the labels per the suggestion from @codedeft 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61165588
  
    True, perhaps we'll need to checkpoint not just the labels but also the data itself for Spark to know how to resume training.  Postponing checkpointing seems like a good idea for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [MLLIB] SPARK-1547: Add Gradient Boosting to M...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the pull request:

    https://github.com/apache/spark/pull/2607#issuecomment-61326184
  
    @manishamde  LGTM!  Thanks for updating the Strategy.  I think this is ready to be merged, though I still plan to update the train* methods to eliminate the ones taking lots of parameters.  In particular, I plan to:
    * Make builder methods for Strategy and Boosting Strategy to make them easy to construct from Java.
    * Eliminate the train* methods taking lots of parameters.
    * Write examples in Scala and Java to make sure everything is easy to do from Java.
    Does that sound OK?
    
    Thanks very much for contributing GBT!  It's a big step forward for MLlib.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org