You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by jkbradley <gi...@git.apache.org> on 2014/09/17 21:45:21 UTC

[GitHub] spark pull request: [SPARK-1545] [mllib] Add Random Forests

GitHub user jkbradley opened a pull request:

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

    [SPARK-1545] [mllib] Add Random Forests

    This PR adds RandomForest to MLlib.  The implementation is basic, and future performance optimizations will be important.  (Note: RFs = Random Forests.)
    
    # Overview
    
    ## RandomForest
    * trains multiple trees at once to reduce the number of passes over the data
    * allows feature subsets at each node
    * uses a queue of nodes instead of fixed groups for each level
    
    This implementation is based an implementation by @manishamde and the [Alpine Labs Sequoia Forest](https://github.com/AlpineNow/SparkML2) by @codedeft (in particular, the TreePoint, BaggedPoint, and node queue implementations).  Thank you for your inputs!
    
    ## Testing
    
    This has been tested for correctness with the test suites and with DecisionTreeRunner on example datasets.
    This has been performance tested using [this branch of spark-perf](https://github.com/jkbradley/spark-perf/tree/rfs).  For training 1 tree, there are small regressions, especially from feature subsampling.
    
    Detailed results are below.  These were run on an EC2 cluster with 15 workers, training 1 tree with maxDepth = 5 (= 6 levels).  The 2 result columns marked with (numTrees) are results after implementing RFs to train multiple trees at once, using a node queue.  The 2 columns marked with (features subsets) are follow-up results after adding feature subsampling.  Speedup values < 1 indicate slowdowns from the old DecisionTree implementation.
    
    numInstances | numFeatures | runtime (sec) | speedup | runtime (sec) | speedup
    ---- | ---- | ---- | ---- | ---- | ----
     | | (numTrees) | (numTrees) | (feature subsets) | (feature subsets)
    20000 | 100 | 4.051 | 1.044433473 | 4.478 | 0.9448414471
    20000 | 500 | 8.472 | 1.104461756 | 9.315 | 1.004508857
    20000 | 1500 | 19.354 | 1.05854087 | 20.863 | 0.9819776638
    20000 | 3500 | 43.674 | 1.072033704 | 45.887 | 1.020332556
    200000 | 100 | 4.196 | 1.171830315 | 4.848 | 1.014232673
    200000 | 500 | 8.926 | 1.082791844 | 9.771 | 0.989151571
    200000 | 1500 | 20.58 | 1.068415938 | 22.134 | 0.9934038131
    200000 | 3500 | 48.043 | 1.075203464 | 52.249 | 0.9886505005
    2000000 | 100 | 4.944 | 1.01355178 | 5.796 | 0.8645617667
    2000000 | 500 | 11.11 | 1.016831683 | 12.482 | 0.9050632911
    2000000 | 1500 | 31.144 | 1.017852556 | 35.274 | 0.8986789136
    2000000 | 3500 | 79.981 | 1.085382778 | 101.105 | 0.8586123337
    20000000 | 100 | 8.304 | 0.9270231214 | 9.073 | 0.8484514494
    20000000 | 500 | 28.174 | 1.083268262 | 34.236 | 0.8914592826
    20000000 | 1500 | 143.97 | 0.9579634646 | 159.275 | 0.8659111599
    
    # Details on specific classes
    
    ## Changes to DecisionTree
    * Main train() method is now in RandomForest.
    * findBestSplits() is no longer needed.  (It split levels into groups, but we now use a queue of nodes.)
    * Many small changes to support RFs.  (Note: These methods should be moved to RandomForest.scala in a later PR, but are in DecisionTree.scala to make code comparison easier.)
    
    ## RandomForest
    * Main train() method is from old DecisionTree.
    * selectNodesToSplit: Note that it selects nodes and feature subsets jointly to track memory usage.
    
    ## RandomForestModel
    * Stores an Array[DecisionTreeModel]
    * Prediction:
     * For classification, most common label.  For regression, mean.
     * We could support other methods later.
    
    ## examples/.../DecisionTreeRunner
    * This now takes numTrees and featureSubsetStrategy, to support RFs.
    
    ## DTStatsAggregator
    * 2 types of functionality (w/ and w/o subsampling features): These require different indexing methods.  (We could treat both as subsampling, but this is less efficient
      DTStatsAggregator is now abstract, and 2 child classes implement these 2 types of functionality.
    
    ## impurities
    * These now take instance weights.
    
    ## Node
    * Some vals changed to vars.
     * This is unfortunately a public API change (DeveloperApi).  This could be avoided by creating a LearningNode struct, but would be awkward.
    
    ## RandomForestSuite
    Please let me know if there are missing tests!
    
    ## BaggedPoint
    This wraps TreePoint and holds bootstrap weights/counts.
    
    # Design decisions
    
    * BaggedPoint: BaggedPoint is separate from TreePoint since it may be useful for other bagging algorithms later on.
    
    * RandomForest public API: What options should be easily supported by the train* methods?  Should ALL options be in the Java-friendly constructors?  Should there be a constructor taking Strategy?
    
    * Feature subsampling options: What options should be supported?  scikit-learn supports the same options, except for "onethird."  One option would be to allow users to specific fractions ("0.1"): the current options could be supported, and any unrecognized values would be parsed as Doubles in [0,1].
    
    * Splits and bins are computed before bootstrapping, so all trees use the same discretization.
    
    * One queue, instead of one queue per tree.
    
    CC: @mengxr @manishamde @codedeft @chouqin  Please let me know if you have suggestions---thanks!


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

    $ git pull https://github.com/jkbradley/spark rfs-new

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

    https://github.com/apache/spark/pull/2435.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 #2435
    
----
commit ac4237808090237fe4c562da8c88c55c330d451f
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T03:17:58Z

    add min info gain and min instances per node parameters in decision tree

commit ff34845c8e43f5b9755dd1fdf428be8b2284c68b
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T04:29:12Z

    separate calculation of predict of node from calculation of info gain

commit 987cbf4b177f29e232bf2ba2ca595ea7015694da
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T04:30:01Z

    fix bug

commit f195e830a94097e5d6d42f22c67c32ca8900d848
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T06:04:20Z

    fix style

commit 845c6fa58c00bfba426e56e71eb46a6f8c3f5985
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T06:05:37Z

    fix style

commit e72c7e4d0ad015fdf25ea2959bdbf524056e38ca
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T06:52:24Z

    add comments

commit 46b891fd7f30b9f2d439134931b35dab387fe2b1
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T08:09:34Z

    fix bug

commit cadd569cf64d6eb7b9c9979a5066a2f63f15fed9
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T08:48:51Z

    add api docs

commit bb465cabc804ca53ef5005f6793b58aa2e4a5274
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T09:09:14Z

    Merge branch 'master' of https://github.com/apache/spark into dt-preprune
    
    Conflicts:
    	mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala

commit 6728fad304511030611c61592b1a590214e7f434
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T09:16:27Z

    minor fix: remove empty lines

commit 10b801269864cda2c00159518688942b1985061b
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T10:10:24Z

    fix style

commit efcc7369f7f52de2810446c6fb976ab1743a63cf
Author: qiping.lqp <qi...@alibaba-inc.com>
Date:   2014-09-09T12:33:37Z

    fix bug

commit 2ab763b2ca1bbc8977777ab898b28965dce5a8a3
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-09T17:42:46Z

    Simplifications to DecisionTree code:
    
    No longer pre-allocate parentImpurities array in main train() method.
    * parentImpurities values are now stored in individual nodes (in Node.stats.impurity).
    
    No longer using Node.build since tree structure is constructed on-the-fly.
    * Did not eliminate since it is public (Developer) API.
    
    Also: Updated DecisionTreeSuite test "Second level node building with vs. without groups"
    * generateOrderedLabeledPoints() modified so that it really does require 2 levels of internal nodes.

commit d593ec70d70b633b72e260c38e89d87ab14fcd69
Author: chouqin <li...@gmail.com>
Date:   2014-09-09T23:57:27Z

    fix docs and change minInstancesPerNode to 1

commit 0278a1198017aae578be3109a8311abc1f9a8e14
Author: chouqin <li...@gmail.com>
Date:   2014-09-10T02:31:57Z

    remove `noSplit` and set `Predict` private to tree

commit 1a8f0add470e4ed53100ce6cf344e24448a0ba42
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T02:34:55Z

    Eliminated pre-allocated nodes array in main train() method.
    * Nodes are constructed and added to the tree structure as needed during training.
    
    Moved tree construction from main train() method into findBestSplitsPerGroup() since there is no need to keep the (split, gain) array for an entire level of nodes.  Only one element of that array is needed at a time, so we do not the array.
    
    findBestSplits() now returns 2 items:
    * rootNode (newly created root node on first iteration, same root node on later iterations)
    * doneTraining (indicating if all nodes at that level were leafs)
    
    Also:
    * Added Node.deepCopy (private[tree]), used for test suite
    * Updated test suite (same functionality)

commit d4dbb99a50418e0168d85db457458d8d96edc242
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T02:35:06Z

    Merge remote-tracking branch 'upstream/master' into dt-spark-3160

commit d4d786407a9bb5fce14dd7999097b21d6fa1cf5e
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T02:45:30Z

    Marked Node.build as deprecated

commit eaa1dcf6a46501779ae58c746e672583d10ff6c8
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T02:58:27Z

    Added topNode doc in DecisionTree and scalastyle fix

commit 306120fc93021f3d2d86333c77296fe3d36b76e1
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T03:09:02Z

    Fixed typo in DecisionTreeModel.scala doc

commit c6e2dfcc62aaa0d26bff90fb34f5b81526ce71c8
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T04:51:35Z

    Added minInstancesPerNode and minInfoGain parameters to DecisionTreeRunner.scala and to Python API in tree.py

commit 39f9b60907050b4e1c78f7413282df13b7e6552c
Author: chouqin <li...@gmail.com>
Date:   2014-09-10T14:15:46Z

    change edge `minInstancesPerNode` to 2 and add one more test

commit c7ebaf1721ba414ed1539bfc4721c3bbfd70b77a
Author: chouqin <li...@gmail.com>
Date:   2014-09-10T14:27:08Z

    fix typo

commit f1d11d15fe519f9ef9d4e1158b309dc6af38864e
Author: chouqin <li...@gmail.com>
Date:   2014-09-10T14:30:22Z

    fix typo

commit 19b01af035719b7e9b67bc85611b4f04b790797a
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T15:52:14Z

    Merge remote-tracking branch 'chouqin/dt-preprune' into chouqin-dt-preprune

commit e2628b605459badb64b8d63059a2821dfff4bd4c
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T23:13:03Z

    Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune

commit 95c479d5a60b166d9c75b9a81cee82e808f23aa0
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-10T23:52:05Z

    * Fixed typo in tree suite test "do not choose split that does not satisfy min instance per node requirements"
    * small style fixes

commit 0dd4d874705643a9d82b9a2a4246a75ba9a7dae9
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-11T01:18:18Z

    Merge remote-tracking branch 'upstream/master' into dt-spark-3160
    
    Conflicts:
    	mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
    	mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala
    	mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala

commit 5c4ac3303fcf94bb5cbbc272013a88ff8c4e7749
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-11T01:26:19Z

    Added check in Strategy to make sure minInstancesPerNode >= 1

commit eddd1eb60e1b63079af2883ad5854fdc22ff07ef
Author: Joseph K. Bradley <jo...@gmail.com>
Date:   2014-09-11T17:54:38Z

    Merge remote-tracking branch 'upstream/master' into rfs-new

----


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56761657
  
    @chouqin  @mengxr  I believe I have addressed your comments, except where I have responded with questions.  Thanks for the feedback!
    @manishamde  I will await more comments---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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56774152
  
    Test failure was in streaming


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943413
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -582,42 +472,36 @@ object DecisionTree extends Serializable with Logging {
          * drastically reduce the communication overhead.
          */
     
    -    // Common calculations for multiple nested methods:
    -
    -    // numNodes:  Number of nodes in this (level of tree, group),
    -    //            where nodes at deeper (larger) levels may be divided into groups.
    -    val numNodes = Node.maxNodesInLevel(level) / numGroups
    +    // numNodes:  Number of nodes in this group
    +    val numNodes = nodesForGroup.values.map(_.size).sum
         logDebug("numNodes = " + numNodes)
     
    +    // Create node index:
    +    //  groupNodeIndex(treeIndex)(node index in tree) = node index in aggregate statistics
    +    //  groupNodeMap(treeIndex)(node index in tree) = node
    +    var idx = 0
    +    val mutableGroupNodeIndex = new mutable.HashMap[Int, Map[Int, Int]]()
    +    val mutableGroupNodeMap = new mutable.HashMap[Int, Map[Int, Node]]()
    +    nodesForGroup.foreach{ case (treeIndex, nodes) =>
    --- End diff --
    
    space before `{`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18064627
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -626,21 +510,30 @@ object DecisionTree extends Serializable with Logging {
          *
          * @param agg  Array storing aggregate calculation, with a set of sufficient statistics for
          *             each (node, feature, bin).
    -     * @param treePoint   Data point being aggregated.
    +     * @param baggedPoint   Data point being aggregated.
          * @return  agg
          */
         def binSeqOp(
             agg: DTStatsAggregator,
    -        treePoint: TreePoint): DTStatsAggregator = {
    -      val nodeIndex = treePointToNodeIndex(treePoint)
    -      // If the example does not reach this level, then nodeIndex < 0.
    -      // If the example reaches this level but is handled in a different group,
    -      //  then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group).
    -      if (nodeIndex >= 0 && nodeIndex < numNodes) {
    -        if (metadata.unorderedFeatures.isEmpty) {
    -          orderedBinSeqOp(agg, treePoint, nodeIndex)
    -        } else {
    -          mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures)
    +        baggedPoint: BaggedPoint[TreePoint]): DTStatsAggregator = {
    +      nodesForGroup.keys.foreach { treeIndex =>
    +        val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures,
    +          bins, metadata.unorderedFeatures)
    +        val aggNodeIndex = groupNodeIndex(treeIndex).getOrElse(nodeIndex, -1)
    --- End diff --
    
    contains and get possibly better than getOrElse and compare >= 0? Definitely more readable.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18102107
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +521,62 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
    -
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
     
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    -        } else {
    -          parentNode.rightNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    nodesForGroup.foreach { case (treeIndex, nodesForTree) =>
    +      nodesForTree.foreach { node =>
    +        val nodeIndex = node.id
    +        val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex)
    +        val aggNodeIndex = nodeInfo.nodeIndexInGroup
    +        val featuresForNode = nodeInfo.featureSubset
    +        val (split: Split, stats: InformationGainStats, predict: Predict) =
    +          binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode)
    +        logDebug("best split = " + split)
    +
    +        // Extract info for this node.  Create children if not leaf.
    +        val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth)
    +        assert(node.id == nodeIndex)
    +        node.predict = predict.predict
    +        node.isLeaf = isLeaf
    +        node.stats = Some(stats)
    +        logDebug("Node = " + node)
    +
    +        if (!isLeaf) {
    +          node.split = Some(split)
    +          node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex)))
    +          node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex)))
    +          nodeQueue.enqueue((treeIndex, node.leftNode.get))
    --- End diff --
    
    Excellent! and a good way to perform local training as well (as you had mentioned earlier). 
    
    cc: @etrain


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-57098176
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20945/consoleFull) for   PR 2435 at commit [`c694174`](https://github.com/apache/spark/commit/c6941748b58f5b77a480cfbc85cdece9ce8dec5a).
     * This patch **passes** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class RandomForestModel(val trees: Array[DecisionTreeModel], val 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18007872
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -582,42 +472,36 @@ object DecisionTree extends Serializable with Logging {
          * drastically reduce the communication overhead.
          */
     
    -    // Common calculations for multiple nested methods:
    -
    -    // numNodes:  Number of nodes in this (level of tree, group),
    -    //            where nodes at deeper (larger) levels may be divided into groups.
    -    val numNodes = Node.maxNodesInLevel(level) / numGroups
    +    // numNodes:  Number of nodes in this group
    +    val numNodes = nodesForGroup.values.map(_.size).sum
         logDebug("numNodes = " + numNodes)
     
    +    // Create node index:
    +    //  groupNodeIndex(treeIndex)(node index in tree) = node index in aggregate statistics
    +    //  groupNodeMap(treeIndex)(node index in tree) = node
    +    var idx = 0
    +    val mutableGroupNodeIndex = new mutable.HashMap[Int, Map[Int, Int]]()
    --- End diff --
    
    Will make a note to try later.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18063221
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,429 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    // TODO: Calculate memory usage more precisely.
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup, featuresForNodes) =
    --- End diff --
    
    Cool.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943424
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    --- End diff --
    
    Shall we use `Long` for random seed to be consistent with Java `Random`?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56319639
  
    @jkbradley thanks, it looks good to me except comments in the code.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56785807
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20786/consoleFull) for   PR 2435 at commit [`ef7c293`](https://github.com/apache/spark/commit/ef7c293838e8ffbea2a847f1ef682dfd7a689fb6).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-61549273
  
    @0asa Thanks. Looks good. Let's move the conversation to the JIRA. 


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-57095923
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20945/consoleFull) for   PR 2435 at commit [`c694174`](https://github.com/apache/spark/commit/c6941748b58f5b77a480cfbc85cdece9ce8dec5a).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943420
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +542,65 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        assert(featuresForNodes.nonEmpty)
    +        new DTStatsAggregatorSubsampledFeatures(metadata, groupNodeIndex, featuresForNodes.get)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
     
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
    -
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    groupNodeIndex.foreach{ case (treeIndex, nodeIndexToAggIndex) =>
    +      nodeIndexToAggIndex.foreach{ case (nodeIndex, aggNodeIndex) =>
    --- End diff --
    
    space before `{`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55979465
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20499/consoleFull) for   PR 2435 at commit [`866e766`](https://github.com/apache/spark/commit/866e7668b687705cd45497671c22774c76f1367b).
     * This patch **passes** 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55964387
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/128/consoleFull) for   PR 2435 at commit [`ff8bb96`](https://github.com/apache/spark/commit/ff8bb967796be47c646bf6ff72eebe960b7d43c7).
     * This patch **fails** 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18051954
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,429 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    // TODO: Calculate memory usage more precisely.
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup, featuresForNodes) =
    --- End diff --
    
    ```nodesForGroup``` and ```featuresForNodes``` are related and could be stored in the same data structure -- map, tuple or a lightweight class.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55974300
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20499/consoleFull) for   PR 2435 at commit [`866e766`](https://github.com/apache/spark/commit/866e7668b687705cd45497671c22774c76f1367b).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943440
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    --- End diff --
    
    remove `{ }`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56897198
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20827/consoleFull) for   PR 2435 at commit [`cc59d78`](https://github.com/apache/spark/commit/cc59d78bc45f5c0b8a0597121c89f932912e2bb3).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18055308
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,429 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    // TODO: Calculate memory usage more precisely.
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup, featuresForNodes) =
    --- End diff --
    
    I agree this part can be simplified.  I'll work on combining some of these mappings.  (There are currently 4 if you include those in findBestSplits, but I think 2 will suffice.  I think 2 is better than 1 so that we don't have to include the mapping to Nodes in the binSeqOp closure.)


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56854191
  
    @jkbradley I will send more comments your way within the next 2 days. Thanks for your patience and sorry for the delay.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

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


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56091137
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20545/consoleFull) for   PR 2435 at commit [`593b13c`](https://github.com/apache/spark/commit/593b13c1af581c11a52ddf4fbf3fe46959a46515).
     * This patch **passes** 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: [SPARK-1545] [mllib] Add Random Forests

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

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


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18074724
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +521,62 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
    -
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
     
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    -        } else {
    -          parentNode.rightNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    nodesForGroup.foreach { case (treeIndex, nodesForTree) =>
    +      nodesForTree.foreach { node =>
    +        val nodeIndex = node.id
    +        val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex)
    +        val aggNodeIndex = nodeInfo.nodeIndexInGroup
    +        val featuresForNode = nodeInfo.featureSubset
    +        val (split: Split, stats: InformationGainStats, predict: Predict) =
    +          binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode)
    +        logDebug("best split = " + split)
    +
    +        // Extract info for this node.  Create children if not leaf.
    +        val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth)
    +        assert(node.id == nodeIndex)
    +        node.predict = predict.predict
    +        node.isLeaf = isLeaf
    +        node.stats = Some(stats)
    +        logDebug("Node = " + node)
    +
    +        if (!isLeaf) {
    +          node.split = Some(split)
    +          node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex)))
    +          node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex)))
    +          nodeQueue.enqueue((treeIndex, node.leftNode.get))
    --- End diff --
    
    I think it's a good idea, and I had actually written up a JIRA (to post after this PR gets in).  I think predictNodeIndex would work fine since it can return nodes in different layers of the tree.  Here's the JIRA:
    
    RandomForest: Learn models too large to store in memory
    
    Proposal: Write trees to disk as they are learned.
    
    RandomForest currently uses a FIFO queue, which means training all trees at once via breadth-first search.  Using a FILO queue would encourage the code to finish one tree before moving on to new ones.  This would allow the code to write trees to disk as they are learned.
    
    Note: It would also be possible to write nodes to disk as they are learned using a FIFO queue, once the example--node mapping is cached [JIRA].  The [Sequoia Forest package]() does this.  However, it could be useful to learn trees progressively, so that future functionality such as early stopping (training fewer trees than expected) could be supported.



---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943458
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    --- End diff --
    
    ditto: `def` -> `override def` and remove doc if it is the same as in the base class
    
    please also update other methods


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18067870
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,7 +170,139 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    --- End diff --
    
    Could probably be in a separate file. It's hard to locate it otherwise.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18065339
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -626,21 +510,30 @@ object DecisionTree extends Serializable with Logging {
          *
          * @param agg  Array storing aggregate calculation, with a set of sufficient statistics for
          *             each (node, feature, bin).
    -     * @param treePoint   Data point being aggregated.
    +     * @param baggedPoint   Data point being aggregated.
          * @return  agg
          */
         def binSeqOp(
             agg: DTStatsAggregator,
    -        treePoint: TreePoint): DTStatsAggregator = {
    -      val nodeIndex = treePointToNodeIndex(treePoint)
    -      // If the example does not reach this level, then nodeIndex < 0.
    -      // If the example reaches this level but is handled in a different group,
    -      //  then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group).
    -      if (nodeIndex >= 0 && nodeIndex < numNodes) {
    -        if (metadata.unorderedFeatures.isEmpty) {
    -          orderedBinSeqOp(agg, treePoint, nodeIndex)
    -        } else {
    -          mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures)
    +        baggedPoint: BaggedPoint[TreePoint]): DTStatsAggregator = {
    +      nodesForGroup.keys.foreach { treeIndex =>
    +        val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures,
    +          bins, metadata.unorderedFeatures)
    +        val aggNodeIndex = groupNodeIndex(treeIndex).getOrElse(nodeIndex, -1)
    --- End diff --
    
    I was trying to be efficient (1 access instead of 2) in this inner loop, though I have not tested both.  (Is it worth careful testing on a cluster?)


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943410
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -582,42 +472,36 @@ object DecisionTree extends Serializable with Logging {
          * drastically reduce the communication overhead.
          */
     
    -    // Common calculations for multiple nested methods:
    -
    -    // numNodes:  Number of nodes in this (level of tree, group),
    -    //            where nodes at deeper (larger) levels may be divided into groups.
    -    val numNodes = Node.maxNodesInLevel(level) / numGroups
    +    // numNodes:  Number of nodes in this group
    +    val numNodes = nodesForGroup.values.map(_.size).sum
         logDebug("numNodes = " + numNodes)
     
    +    // Create node index:
    +    //  groupNodeIndex(treeIndex)(node index in tree) = node index in aggregate statistics
    +    //  groupNodeMap(treeIndex)(node index in tree) = node
    +    var idx = 0
    +    val mutableGroupNodeIndex = new mutable.HashMap[Int, Map[Int, Int]]()
    --- End diff --
    
    minor: Later we can try to use `spark.util.collection.OpenHashMap` instead for memory and speed.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943435
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    +          featuresForNodes: Option[Map[Int, Map[Int, Array[Int]]]]) =
    +        RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng)
    +      // Sanity check (should never occur):
    +      assert(nodesForGroup.size > 0,
    +        s"RandomForest selected empty nodesForGroup.  Error for unknown reason.")
    +
    +      // Choose node splits, and enqueue new nodes as needed.
    +      timer.start("findBestSplits")
    +      DecisionTree.findBestSplits(baggedInput,
    +        metadata, topNodes, nodesForGroup, featuresForNodes, splits, bins, nodeQueue, timer)
    +      timer.stop("findBestSplits")
    +    }
    +
    +    timer.stop("total")
    +
    +    logInfo("Internal timing for DecisionTree:")
    +    logInfo(s"$timer")
    +
    +    val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo))
    +    RandomForestModel.build(trees)
    +  }
    +
    +}
    +
    +object RandomForest extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Classification,
    +      s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param numClassesForClassification number of classes for classification.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "gini" (recommended) or "entropy".
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Classification, impurityType, maxDepth,
    +      numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo)
    +    trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]]
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainClassifier(input.rdd, numClassesForClassification,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Regression,
    +      s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "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.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Regression, impurityType, maxDepth,
    +      0, maxBins, Sort, categoricalFeaturesInfo)
    +    trainRegressor(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]]
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainRegressor(input.rdd,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * List of supported feature subset sampling strategies.
    +   */
    +  val supportedFeatureSubsetStrategies: Array[String] =
    +    Array("auto", "all", "sqrt", "log2", "onethird")
    +
    +  /**
    +   * Pull nodes off of the queue to split.
    +   * Track memory usage for aggregates, and stop adding nodes when too large.
    +   * @param nodeQueue  Queue of nodes to split.
    +   * @param maxMemoryUsage  Bound on size of aggregate statistics.
    +   * @return  (nodesForGroup, featuresForNodes).
    +   *          nodesForGroup holds the nodes to split: treeIndex --> nodes in tree.
    +   *          featuresForNodes holds selected features for each node:
    +   *            treeIndex --> node index --> feature indices.
    +   *          featuresForNodes is only used when subsampling features;
    +   *          it is empty if not subsampling.
    +   */
    +  private[tree] def selectNodesToSplit(
    +      nodeQueue: mutable.Queue[(Int, Node)],
    +      maxMemoryUsage: Long,
    +      metadata: DecisionTreeMetadata,
    +      rng: scala.util.Random): (Map[Int, Array[Node]], Option[Map[Int, Map[Int, Array[Int]]]]) = {
    +    // Collect some nodes to split:
    +    //  nodesForGroup(treeIndex) = nodes to split
    +    val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]()
    +    val mutableFeaturesForNodes = new mutable.HashMap[Int, mutable.HashMap[Int, Array[Int]]]()
    +    var memUsage: Long = 0L
    +    while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) {
    +      val (treeIndex, node) = nodeQueue.head
    +      // Choose subset of features for node (if subsampling).
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // TODO: Use more efficient subsampling?
    +        Some(rng.shuffle(Range(0, metadata.numFeatures).toList)
    +          .take(metadata.numFeaturesPerNode).toArray)
    +      } else {
    +        None
    +      }
    +      val nodeMemUsage = RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +      if (memUsage + nodeMemUsage <= maxMemoryUsage) {
    +        nodeQueue.dequeue()
    +        mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[Node]()) += node
    +        if (featureSubset.nonEmpty) {
    +          mutableFeaturesForNodes
    +            .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, Array[Int]]())(node.id)
    +            = featureSubset.get
    +        }
    +      }
    +      memUsage += nodeMemUsage
    +    }
    +    val nodesForGroup: Map[Int, Array[Node]] = mutableNodesForGroup.mapValues(_.toArray).toMap
    +    val featuresForNodes = if (metadata.subsamplingFeatures) {
    +      Some(mutableFeaturesForNodes.mapValues(_.toMap).toMap)
    +    } else {
    +      None
    +    }
    +    (nodesForGroup, featuresForNodes)
    +  }
    +
    +  /**
    +   * Get the number of values to be stored for this node in the bin aggregates.
    +   * @param featureSubset  Indices of features which may be split at this node.
    +   *                       If None, then use all features.
    +   */
    +  private[tree] def numElementsForNode(
    --- End diff --
    
    `numElementsForNode` -> `aggSizeForNode`?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943415
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -582,42 +472,36 @@ object DecisionTree extends Serializable with Logging {
          * drastically reduce the communication overhead.
          */
     
    -    // Common calculations for multiple nested methods:
    -
    -    // numNodes:  Number of nodes in this (level of tree, group),
    -    //            where nodes at deeper (larger) levels may be divided into groups.
    -    val numNodes = Node.maxNodesInLevel(level) / numGroups
    +    // numNodes:  Number of nodes in this group
    +    val numNodes = nodesForGroup.values.map(_.size).sum
         logDebug("numNodes = " + numNodes)
     
    +    // Create node index:
    +    //  groupNodeIndex(treeIndex)(node index in tree) = node index in aggregate statistics
    +    //  groupNodeMap(treeIndex)(node index in tree) = node
    +    var idx = 0
    +    val mutableGroupNodeIndex = new mutable.HashMap[Int, Map[Int, Int]]()
    +    val mutableGroupNodeMap = new mutable.HashMap[Int, Map[Int, Node]]()
    +    nodesForGroup.foreach{ case (treeIndex, nodes) =>
    +      val nodeIndexToAggIndex = new mutable.HashMap[Int, Int]()
    +      val nodeIndexToNode = new mutable.HashMap[Int, Node]()
    +      nodes.foreach{ node =>
    --- End diff --
    
    space before `{`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18065224
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -465,43 +329,60 @@ object DecisionTree extends Serializable with Logging {
        * @param agg  Array storing aggregate calculation, with a set of sufficient statistics for
        *             each (node, feature, bin).
        * @param treePoint  Data point being aggregated.
    -   * @param nodeIndex  Node corresponding to treePoint. Indexed from 0 at start of (level, group).
    +   * @param nodeIndex  Node corresponding to treePoint.  agg is indexed in [0, numNodes).
        * @param bins possible bins for all features, indexed (numFeatures)(numBins)
        * @param unorderedFeatures  Set of indices of unordered features.
    +   * @param instanceWeight  Weight (importance) of instance in dataset.
        */
       private def mixedBinSeqOp(
           agg: DTStatsAggregator,
           treePoint: TreePoint,
           nodeIndex: Int,
           bins: Array[Array[Bin]],
    -      unorderedFeatures: Set[Int]): Unit = {
    -    // Iterate over all features.
    -    val numFeatures = treePoint.binnedFeatures.size
    +      unorderedFeatures: Set[Int],
    +      instanceWeight: Double,
    +      featuresForNode: Option[Array[Int]]): Unit = {
    +    val numFeaturesPerNode = if (featuresForNode.nonEmpty) {
    +      // Use subsampled features
    +      featuresForNode.get.size
    +    } else {
    +      // Use all features
    +      agg.metadata.numFeatures
    +    }
         val nodeOffset = agg.getNodeOffset(nodeIndex)
    -    var featureIndex = 0
    -    while (featureIndex < numFeatures) {
    +    // Iterate over features.
    +    var featureIndexIdx = 0
    +    while (featureIndexIdx < numFeaturesPerNode) {
    +      val featureIndex = if (featuresForNode.nonEmpty) {
    +        featuresForNode.get.apply(featureIndexIdx)
    --- End diff --
    
    I agree.  But I felt that featuresForNode.get(featureIndexIdx) would be weird since it makes it look like featureIndexIdx is an argument of get.  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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18070843
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +521,62 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
    -
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
     
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    -        } else {
    -          parentNode.rightNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    nodesForGroup.foreach { case (treeIndex, nodesForTree) =>
    +      nodesForTree.foreach { node =>
    +        val nodeIndex = node.id
    +        val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex)
    +        val aggNodeIndex = nodeInfo.nodeIndexInGroup
    +        val featuresForNode = nodeInfo.featureSubset
    +        val (split: Split, stats: InformationGainStats, predict: Predict) =
    +          binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode)
    +        logDebug("best split = " + split)
    +
    +        // Extract info for this node.  Create children if not leaf.
    +        val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth)
    +        assert(node.id == nodeIndex)
    +        node.predict = predict.predict
    +        node.isLeaf = isLeaf
    +        node.stats = Some(stats)
    +        logDebug("Node = " + node)
    +
    +        if (!isLeaf) {
    +          node.split = Some(split)
    +          node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex)))
    +          node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex)))
    +          nodeQueue.enqueue((treeIndex, node.leftNode.get))
    --- End diff --
    
    Well, it might not be possible to do a DFS with the current ```predictNodeIndex``` method since we assume data point contributes to one node in a tree.  
    
    Having said that, is there a corner case where nodes for the same tree on two different layers could be a part of the same group during BFS?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943442
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    +
    +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
    +   */
    +  def convertToBaggedRDD[Datum](
    +      input: RDD[Datum],
    +      numSubsamples: Int,
    +      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    --- End diff --
    
    use `Long` for random seed


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18041353
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-57099927
  
    LGTM! Thanks @jkbradley for adding the highly requested RF feature. 


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943476
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala ---
    @@ -0,0 +1,106 @@
    +/*
    + * 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.rdd.RDD
    +
    +import scala.collection.mutable
    --- End diff --
    
    organize imports


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943471
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- End diff --
    
    Do we have any reference for `log2` and `onethird`?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943480
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala ---
    @@ -0,0 +1,106 @@
    +/*
    + * 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.rdd.RDD
    +
    +import scala.collection.mutable
    +
    +/**
    + * :: Experimental ::
    + * Random forest model for classification or regression.
    + * This model stores a collection of [[DecisionTreeModel]] instances and uses them to make
    + * aggregate predictions.
    + * @param trees Trees which make up this forest.  This cannot be empty.
    + * @param algo algorithm type -- classification or regression
    + */
    +@Experimental
    +class RandomForestModel(private[tree] val trees: Array[DecisionTreeModel], val algo: Algo)
    --- End diff --
    
    make `trees` public?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18046525
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    --- End diff --
    
    Ok. There might be a scenario where one might train multiple trees without bagging but with different split selection strategies but I think we can handle it in the future.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56992076
  
    Note: I tested using Spark's OpenHashMap (org.apache.spark.util.collection.OpenHashMap) for the 2 node group maps (nodesForGroup, treeToNodeToIndexInfo), and it made no difference.  I will stick with the Scala ones.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17764131
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   * @param nodeOffset  Pre-computed node offset from [[getNodeOffset]].
    +   */
    +  def nodeUpdate(
    +      nodeOffset: Int,
    +      nodeIndex: Int,
    --- End diff --
    
    `nodeIndex` is never used, because we already have `nodeOffset`, which is computed by `nodeIndex`.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55967377
  
    Hi Joseph,
    
    I'll take a look when I can, but this is a massive PR, so I'm not sure if
    I'll have time to go through this thoroughly.
    
    * I suppose that from Alpine's point of view, we should then wait until
    this is merged before contributing local training and others?
    * Does this also include node caching (instead of passing trees to
    executors) ?
    
    Thanks!
    
    
    On Wed, Sep 17, 2014 at 12:45 PM, jkbradley <no...@github.com>
    wrote:
    
    > This PR adds RandomForest to MLlib. The implementation is basic, and
    > future performance optimizations will be important. (Note: RFs = Random
    > Forests.)
    > Overview RandomForest
    >
    >    - trains multiple trees at once to reduce the number of passes over
    >    the data
    >    - allows feature subsets at each node
    >    - uses a queue of nodes instead of fixed groups for each level
    >
    > This implementation is based an implementation by @manishamde
    > <https://github.com/manishamde> and the Alpine Labs Sequoia Forest
    > <https://github.com/AlpineNow/SparkML2> by @codedeft
    > <https://github.com/codedeft> (in particular, the TreePoint, BaggedPoint,
    > and node queue implementations). Thank you for your inputs!
    > Testing
    >
    > This has been tested for correctness with the test suites and with
    > DecisionTreeRunner on example datasets.
    > This has been performance tested using this branch of spark-perf
    > <https://github.com/jkbradley/spark-perf/tree/rfs>. For training 1 tree,
    > there are small regressions, especially from feature subsampling.
    >
    > Detailed results are below. These were run on an EC2 cluster with 15
    > workers, training 1 tree with maxDepth = 5 (= 6 levels). The 2 result
    > columns marked with (numTrees) are results after implementing RFs to train
    > multiple trees at once, using a node queue. The 2 columns marked with
    > (features subsets) are follow-up results after adding feature subsampling.
    > Speedup values < 1 indicate slowdowns from the old DecisionTree
    > implementation.
    >  numInstances numFeatures runtime (sec) speedup runtime (sec) speedup
    > (numTrees) (numTrees) (feature subsets) (feature subsets)  20000 100 4.051
    > 1.044433473 4.478 0.9448414471  20000 500 8.472 1.104461756 9.315
    > 1.004508857  20000 1500 19.354 1.05854087 20.863 0.9819776638  20000 3500
    > 43.674 1.072033704 45.887 1.020332556  200000 100 4.196 1.171830315 4.848
    > 1.014232673  200000 500 8.926 1.082791844 9.771 0.989151571  200000 1500
    > 20.58 1.068415938 22.134 0.9934038131  200000 3500 48.043 1.075203464
    > 52.249 0.9886505005  2000000 100 4.944 1.01355178 5.796 0.8645617667
    > 2000000 500 11.11 1.016831683 12.482 0.9050632911  2000000 1500 31.144
    > 1.017852556 35.274 0.8986789136  2000000 3500 79.981 1.085382778 101.105
    > 0.8586123337  20000000 100 8.304 0.9270231214 9.073 0.8484514494  20000000
    > 500 28.174 1.083268262 34.236 0.8914592826  20000000 1500 143.97
    > 0.9579634646 159.275 0.8659111599  Details on specific classes Changes to
    > DecisionTree
    >
    >    - Main train() method is now in RandomForest.
    >    - findBestSplits() is no longer needed. (It split levels into groups,
    >    but we now use a queue of nodes.)
    >    - Many small changes to support RFs. (Note: These methods should be
    >    moved to RandomForest.scala in a later PR, but are in DecisionTree.scala to
    >    make code comparison easier.)
    >
    > RandomForest
    >
    >    - Main train() method is from old DecisionTree.
    >    - selectNodesToSplit: Note that it selects nodes and feature subsets
    >    jointly to track memory usage.
    >
    > RandomForestModel
    >
    >    - Stores an Array[DecisionTreeModel]
    >    - Prediction:
    >       - For classification, most common label. For regression, mean.
    >       - We could support other methods later.
    >
    > examples/.../DecisionTreeRunner
    >
    >    - This now takes numTrees and featureSubsetStrategy, to support RFs.
    >
    > DTStatsAggregator
    >
    >    - 2 types of functionality (w/ and w/o subsampling features): These
    >    require different indexing methods. (We could treat both as subsampling,
    >    but this is less efficient DTStatsAggregator is now abstract, and 2 child
    >    classes implement these 2 types of functionality.
    >
    > impurities
    >
    >    - These now take instance weights.
    >
    > Node
    >
    >    - Some vals changed to vars.
    >       - This is unfortunately a public API change (DeveloperApi). This
    >       could be avoided by creating a LearningNode struct, but would be awkward.
    >
    > RandomForestSuite
    >
    > Please let me know if there are missing tests!
    > BaggedPoint
    >
    > This wraps TreePoint and holds bootstrap weights/counts.
    > Design decisions
    >
    >    -
    >
    >    BaggedPoint: BaggedPoint is separate from TreePoint since it may be
    >    useful for other bagging algorithms later on.
    >    -
    >
    >    RandomForest public API: What options should be easily supported by
    >    the train* methods? Should ALL options be in the Java-friendly
    >    constructors? Should there be a constructor taking Strategy?
    >    -
    >
    >    Feature subsampling options: What options should be supported?
    >    scikit-learn supports the same options, except for "onethird." One option
    >    would be to allow users to specific fractions ("0.1"): the current options
    >    could be supported, and any unrecognized values would be parsed as Doubles
    >    in [0,1].
    >    -
    >
    >    Splits and bins are computed before bootstrapping, so all trees use
    >    the same discretization.
    >    -
    >
    >    One queue, instead of one queue per tree.
    >
    > CC: @mengxr <https://github.com/mengxr> @manishamde
    > <https://github.com/manishamde> @codedeft <https://github.com/codedeft>
    > @chouqin <https://github.com/chouqin> Please let me know if you have
    > suggestions---thanks!
    > ------------------------------
    > You can merge this Pull Request by running
    >
    >   git pull https://github.com/jkbradley/spark rfs-new
    >
    > Or view, comment on, or merge it at:
    >
    >   https://github.com/apache/spark/pull/2435
    > Commit Summary
    >
    >    - add min info gain and min instances per node parameters in decision
    >    tree
    >    - separate calculation of predict of node from calculation of info gain
    >    - fix bug
    >    - fix style
    >    - fix style
    >    - add comments
    >    - fix bug
    >    - add api docs
    >    - Merge branch 'master' of https://github.com/apache/spark into
    >    dt-preprune
    >    - minor fix: remove empty lines
    >    - fix style
    >    - fix bug
    >    - Simplifications to DecisionTree code:
    >    - fix docs and change minInstancesPerNode to 1
    >    - remove `noSplit` and set `Predict` private to tree
    >    - Eliminated pre-allocated nodes array in main train() method.
    >    - Merge remote-tracking branch 'upstream/master' into dt-spark-3160
    >    - Marked Node.build as deprecated
    >    - Added topNode doc in DecisionTree and scalastyle fix
    >    - Fixed typo in DecisionTreeModel.scala doc
    >    - Added minInstancesPerNode and minInfoGain parameters to
    >    DecisionTreeRunner.scala and to Python API in tree.py
    >    - change edge `minInstancesPerNode` to 2 and add one more test
    >    - fix typo
    >    - fix typo
    >    - Merge remote-tracking branch 'chouqin/dt-preprune' into
    >    chouqin-dt-preprune
    >    - Merge remote-tracking branch 'upstream/master' into
    >    chouqin-dt-preprune
    >    - * Fixed typo in tree suite test "do not choose split that does not
    >    satisfy min instance per node requirements"
    >    - Merge remote-tracking branch 'upstream/master' into dt-spark-3160
    >    - Added check in Strategy to make sure minInstancesPerNode >= 1
    >    - Merge remote-tracking branch 'upstream/master' into rfs-new
    >    - RFs partly implemented, not done yet
    >    - Merge remote-tracking branch 'upstream/master' into
    >    chouqin-dt-preprune
    >    - Added max of 10GB for maxMemoryInMB in Strategy.
    >    - Merge remote-tracking branch 'upstream/master' into rfs-new
    >    - Merge branch 'chouqin-dt-preprune' into rfs-new
    >    - Basic random forests are implemented. Random features per node not
    >    yet implemented. Test suite not implemented.
    >    - Updated docs. Small fix for bug which does not cause errors: No
    >    longer allocate unused child nodes for leaf nodes.
    >    - Merge remote-tracking branch 'upstream/master' into rfs-new. Added
    >    RandomForestModel.toString
    >    - Implemented feature subsampling. Tested DecisionTree but not
    >    RandomForest.
    >    - Added numTrees and featureSubsetStrategy to DecisionTreeRunner (to
    >    support RandomForest). Fixed bugs so that RandomForest now runs.
    >    - Added RandomForestSuite, and fixed small bugs, style issues.
    >    - Merge remote-tracking branch 'upstream/master' into rfs-new
    >    - removed usage of null from RandomForest and replaced with Option
    >
    > File Changes
    >
    >    - *M*
    >    examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-0> (82)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-1> (477)
    >    - *A*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-2> (430)
    >    - *A*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-3> (80)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-4> (311)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-5> (38)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-6> (4)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-7> (4)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-8> (2)
    >    - *M*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-9> (8)
    >    - *M* mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-10> (13)
    >    - *A*
    >    mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-11> (106)
    >    - *M*
    >    mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-12> (203)
    >    - *A*
    >    mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala
    >    <https://github.com/apache/spark/pull/2435/files#diff-13> (219)
    >
    > Patch Links:
    >
    >    - https://github.com/apache/spark/pull/2435.patch
    >    - https://github.com/apache/spark/pull/2435.diff
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/2435>.
    >


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18046081
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18014914
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- End diff --
    
    OK, I'll keep 1/3.  Would you recommend using a warning when log2 or sqrt are used?  Or I could remove log2, and make the user specify something that strict manually (assuming we later add support for parsing arbitrary numerical values given for this parameter).


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17957565
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    --- End diff --
    
    I started to add comments, but I realized I was making a 3rd copy of the doc (which was already in selectNodesToSplit and in findBestSplits).  I think I will remove the explicit types and not add doc (since the current comment in the line above says what nodesForGroup, featuresForNodes mean at a high level).


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943465
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   * @param nodeOffset  Pre-computed node offset from [[getNodeOffset]].
    +   */
    +  def nodeUpdate(
    +      nodeOffset: Int,
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]].
    +   * For ordered features only.
    +   */
    +  def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = {
    +    require(!isUnordered(featureIndex),
    +      s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" +
    +        s" for unordered feature $featureIndex.")
    +    nodeIndex * nodeStride + featureOffsets(featureIndex)
    +  }
    +
    +  /**
    +   * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]].
    +   * For unordered features only.
    +   */
    +  def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = {
    +    require(isUnordered(featureIndex),
    --- End diff --
    
    +1


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55976264
  
    I'll make a JIRA for supporting hand-picked numbers of features; we can discuss fraction vs. integer there.  I like the functional options (sqrt, log2) supported by scikit-learn since those are arguably sort of data-adaptive, but I agree hand-picked values will be necessary.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17764228
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   * @param nodeOffset  Pre-computed node offset from [[getNodeOffset]].
    +   */
    +  def nodeUpdate(
    +      nodeOffset: Int,
    +      nodeIndex: Int,
    --- End diff --
    
    sorry, `nodeIndex` is needed in `DTStatsAggregatorSubsampledFeatures`. Close this comment.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943398
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -17,20 +17,21 @@
     
     package org.apache.spark.examples.mllib
     
    +import org.apache.spark.util.Utils
    --- End diff --
    
    organize imports


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-61494531
  
    @0asa Yes. PRs for these will be great. 
    
    Could you check if there are already existing JIRA for these -- if not, you could create a JIRA tickets. Also, please make sure these tickets are assigned to you before working on it to avoid duplicate 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18019000
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- End diff --
    
    The `log2` is from Breiman's paper: http://www.stat.berkeley.edu/~breiman/randomforest2001.pdf
    
    From R's randomForest doc:
    > Note that the default values are different for classification (sqrt(p) where p is number of
    variables in x) and regression (p/3)
    
    From http://www.stat.berkeley.edu/~breiman/Using_random_forests_V3.1.pdf
    > this is the only parameter that requires some judgment to set, but
    forests isn't too sensitive to its value as long as it's in the right ball
    park. I have found that setting mtry equal to the square root of
    mdim gives generally near optimum results. My advice is to begin
    with this value and try a value twice as high and half as low
    monitoring the results by setting look=1 and checking the internal
    test set error for a small number of trees. With many noise
    variables present, mtry has to be set higher.
    
    Let's set the default to `sqrt`, keep `log2` and `onethird`, and mention the references in the doc or comments.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17765427
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    --- End diff --
    
    In `DTStatsAggregatorFixedFeatures` and `DTStatsAggregatorSubsampledFeatures`, `impurityAggregator.update` is called with `_allStats`, and in `DTStatsAggregator` it is called with `allStats`. I think we make it consistent by making a function `updateAggregator` in `DTStatsAggregator` like this:
    
    ```
    def updateAggregator(offset: Int, label: Double, instanceWeight: Double): Unit = {
        impurityAggregator.update(allStats, offset, label, instanceWeight)
      }
    ```
    
    all other functions that need to call `impurityAggragator.update` call this function.
    
    



---
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: [SPARK-1545] [mllib] Add Random Forests

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

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


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56246602
  
    @chouqin  Thanks for the feedback!  I'll test the update* variants to see if they really are more efficient, and will work on the other points 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55976349
  
    For naming, scikit-learn uses "max_features" instead of "featureSubsetStrategy".  Both of those are a little vague.  I'm wondering if the name should be changed to "maxFeaturesPerNode"


---
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: [SPARK-1545] [mllib] Add Random Forests

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

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


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56903067
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20826/consoleFull) for   PR 2435 at commit [`e25909f`](https://github.com/apache/spark/commit/e25909fb4cc655b1cd597899364f47d06067b93d).
     * This patch **passes** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class RandomForestModel(val trees: Array[DecisionTreeModel], val 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17889650
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    --- End diff --
    
    Ok, I see the TODO a few lines below. Maybe, we should move it 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17765048
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    --- End diff --
    
    just curious, is this function really faster than `update`. I think it just saves one multiplication and in `DTStatsAggregatorSubsampledFeatures` it saves nothing.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18070099
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +521,62 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
    -
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
     
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    -        } else {
    -          parentNode.rightNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    nodesForGroup.foreach { case (treeIndex, nodesForTree) =>
    +      nodesForTree.foreach { node =>
    +        val nodeIndex = node.id
    +        val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex)
    +        val aggNodeIndex = nodeInfo.nodeIndexInGroup
    +        val featuresForNode = nodeInfo.featureSubset
    +        val (split: Split, stats: InformationGainStats, predict: Predict) =
    +          binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode)
    +        logDebug("best split = " + split)
    +
    +        // Extract info for this node.  Create children if not leaf.
    +        val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth)
    +        assert(node.id == nodeIndex)
    +        node.predict = predict.predict
    +        node.isLeaf = isLeaf
    +        node.stats = Some(stats)
    +        logDebug("Node = " + node)
    +
    +        if (!isLeaf) {
    +          node.split = Some(split)
    +          node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex)))
    +          node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex)))
    +          nodeQueue.enqueue((treeIndex, node.leftNode.get))
    --- End diff --
    
    Discussion: We are doing a BFS here. There might be merit in doing a comparison with DFS. We might be able to get to local single-machine computations quicker. Not sure whether it's a good thing or not. :-) There might be other repercussions we might have not foreseen. Finally, we should expect the same answer by BFS and DFS so it will be a good bug check. :-)


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943446
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    +
    +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
    +   */
    +  def convertToBaggedRDD[Datum](
    +      input: RDD[Datum],
    +      numSubsamples: Int,
    +      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    +    input.mapPartitionsWithIndex{ (partitionIndex, instances) =>
    +      // TODO: Support different sampling rates, and sampling without replacement.
    +      // Use random seed = seed + partitionIndex + 1 to make generation reproducible.
    +      val poisson = new Poisson(1.0, new DRand(seed + partitionIndex + 1))
    +      instances.map{ instance =>
    --- End diff --
    
    space before `{`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18008890
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- End diff --
    
    sqrt and log2 are used in scikit-learn.  I have not seen onethird.  I'd be OK with removing it.  (Can you also please check out the comments above about this?  It would be good to know if I should go ahead and include more options for this parameter, and also if I should rename the parameter to match another library.  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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17889907
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    --- End diff --
    
    Minor: One could remove the explicit types and add a comment to explain what data each variable holds.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56895492
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20826/consoleFull) for   PR 2435 at commit [`e25909f`](https://github.com/apache/spark/commit/e25909fb4cc655b1cd597899364f47d06067b93d).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18074749
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +521,62 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
    -
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
     
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    -        } else {
    -          parentNode.rightNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    nodesForGroup.foreach { case (treeIndex, nodesForTree) =>
    +      nodesForTree.foreach { node =>
    +        val nodeIndex = node.id
    +        val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex)
    +        val aggNodeIndex = nodeInfo.nodeIndexInGroup
    +        val featuresForNode = nodeInfo.featureSubset
    +        val (split: Split, stats: InformationGainStats, predict: Predict) =
    +          binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode)
    +        logDebug("best split = " + split)
    +
    +        // Extract info for this node.  Create children if not leaf.
    +        val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth)
    +        assert(node.id == nodeIndex)
    +        node.predict = predict.predict
    +        node.isLeaf = isLeaf
    +        node.stats = Some(stats)
    +        logDebug("Node = " + node)
    +
    +        if (!isLeaf) {
    +          node.split = Some(split)
    +          node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex)))
    +          node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex)))
    +          nodeQueue.enqueue((treeIndex, node.leftNode.get))
    --- End diff --
    
    Oh, and with respect to nodes in different layers, yes that can happen.  But it should not be a problem since we now allocate nodes as they are needed, rather than for a whole level at once.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943417
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -649,71 +542,65 @@ object DecisionTree extends Serializable with Logging {
         // Calculate bin aggregates.
         timer.start("aggregation")
         val binAggregates: DTStatsAggregator = {
    -      val initAgg = new DTStatsAggregator(metadata, numNodes)
    +      val initAgg = if (metadata.subsamplingFeatures) {
    +        assert(featuresForNodes.nonEmpty)
    +        new DTStatsAggregatorSubsampledFeatures(metadata, groupNodeIndex, featuresForNodes.get)
    +      } else {
    +        new DTStatsAggregatorFixedFeatures(metadata, numNodes)
    +      }
           input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp)
         }
         timer.stop("aggregation")
     
    -    // Calculate best splits for all nodes at a given level
    +    // Calculate best splits for all nodes in the group
         timer.start("chooseSplits")
    -    // On the first iteration, we need to get and return the newly created root node.
    -    var newTopNode: Node = topNode
    -
    -    // Iterate over all nodes at this level
    -    var nodeIndex = 0
    -    var internalNodeCount = 0
    -    while (nodeIndex < numNodes) {
    -      val (split: Split, stats: InformationGainStats, predict: Predict) =
    -        binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits)
    -      logDebug("best split = " + split)
     
    -      val globalNodeIndex = globalNodeIndexOffset + nodeIndex
    -
    -      // Extract info for this node at the current level.
    -      val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth)
    -      val node =
    -        new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats))
    -      logDebug("Node = " + node)
    -
    -      if (!isLeaf) {
    -        internalNodeCount += 1
    -      }
    -      if (level == 0) {
    -        newTopNode = node
    -      } else {
    -        // Set parent.
    -        val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode)
    -        if (Node.isLeftChild(globalNodeIndex)) {
    -          parentNode.leftNode = Some(node)
    +    // Iterate over all nodes in this group.
    +    groupNodeIndex.foreach{ case (treeIndex, nodeIndexToAggIndex) =>
    --- End diff --
    
    space before `{`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17957666
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    --- End diff --
    
    I was curious too.  I just ran some experiments on EC2.  With 1 worker, there is basically no difference.  With 16 workers, there is a difference when there are lots of ordered features (where this function nodeUpdate is used): eliminating nodeUpdate and using update makes things run about 5% slower.  I will keep it 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943444
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    +
    +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
    +   */
    +  def convertToBaggedRDD[Datum](
    +      input: RDD[Datum],
    +      numSubsamples: Int,
    +      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    +    input.mapPartitionsWithIndex{ (partitionIndex, instances) =>
    --- End diff --
    
    space before `{`


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943501
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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.mutable
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.linalg.Vectors
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata}
    +import org.apache.spark.mllib.tree.impurity.Gini
    +import org.apache.spark.mllib.tree.model.{Node, RandomForestModel}
    +import org.apache.spark.mllib.util.LocalSparkContext
    +import org.apache.spark.util.StatCounter
    +
    +/**
    + * Test suite for [[RandomForest]].
    + */
    +class RandomForestSuite extends FunSuite with LocalSparkContext {
    +
    +  test("BaggedPoint RDD: without subsampling") {
    +    val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1)
    +    val rdd = sc.parallelize(arr)
    +    val baggedRDD = BaggedPoint.convertToBaggedRDDWithoutSampling(rdd)
    +    baggedRDD.collect().foreach { baggedPoint =>
    +      assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 1.0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, numSubsamples, seed = seed)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      RandomForestSuite.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev,
    +        epsilon = 0.01)
    +    }
    +  }
    +
    +  test("Binary classification with continuous features:" +
    +      " comparing DecisionTree vs. RandomForest(numTrees = 1)") {
    +
    +    val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +    val numTrees = 1
    +
    +    val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    val rf = RandomForest.trainClassifier(rdd, strategy, numTrees = numTrees,
    +      featureSubsetStrategy = "auto", seed = 123)
    +    assert(rf.trees.size === 1)
    +    val rfTree = rf.trees(0)
    +
    +    val dt = DecisionTree.train(rdd, strategy)
    +
    +    RandomForestSuite.validateClassifier(rf, arr, 0.9)
    +    DecisionTreeSuite.validateClassifier(dt, arr, 0.9)
    +
    +    // Make sure trees are the same.
    +    assert(rfTree.toString == dt.toString)
    +  }
    +
    +  test("Binary classification with continuous features: subsampling features") {
    +    val numFeatures = 50
    +    val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures)
    +    val rdd = sc.parallelize(arr)
    +    val categoricalFeaturesInfo = Map.empty[Int, Int]
    +
    +    val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2,
    +      numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo)
    +
    +    // Select feature subset for top nodes.  Return true if OK.
    +    def checkFeatureSubsetStrategy(
    +        numTrees: Int,
    +        featureSubsetStrategy: String,
    +        numFeaturesPerNode: Int): Unit = {
    +      val seeds = Array(123, 5354, 230, 349867, 23987)
    +      val maxMemoryUsage: Long = 128 * 1024L * 1024L
    +      val metadata =
    +        DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees, featureSubsetStrategy)
    +      seeds.foreach { seed =>
    +        val failString = s"Failed on test with:" +
    +          s"numTrees=$numTrees, featureSubsetStrategy=$featureSubsetStrategy," +
    +          s" numFeaturesPerNode=$numFeaturesPerNode, seed=$seed"
    +        val nodeQueue = new mutable.Queue[(Int, Node)]()
    +        val topNodes: Array[Node] = new Array[Node](numTrees)
    +        Range(0, numTrees).foreach { treeIndex =>
    +          topNodes(treeIndex) = Node.emptyNode(nodeIndex = 1)
    +          nodeQueue.enqueue((treeIndex, topNodes(treeIndex)))
    +        }
    +        val rng = new scala.util.Random(seed = seed)
    +        val (nodesForGroup: Map[Int, Array[Node]],
    +        featuresForNodes: Option[Map[Int, Map[Int, Array[Int]]]]) =
    +          RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng)
    +
    +        assert(nodesForGroup.size === numTrees, failString)
    +        assert(nodesForGroup.values.forall(_.size == 1), failString) // 1 node per tree
    +        if (numFeaturesPerNode == numFeatures) {
    +          assert(featuresForNodes.isEmpty, failString)
    +        } else {
    +          // Check number of features.
    +          featuresForNodes.get.foreach { case (treeIndex, nodeToFeaturesMap) =>
    +            nodeToFeaturesMap.foreach { case (nodeIndex, features) =>
    +              assert(features.size === numFeaturesPerNode, failString)
    +            }
    +          }
    +        }
    +      }
    +    }
    +
    +    checkFeatureSubsetStrategy(numTrees = 1, "auto", numFeatures)
    +    checkFeatureSubsetStrategy(numTrees = 1, "all", numFeatures)
    +    checkFeatureSubsetStrategy(numTrees = 1, "sqrt", math.sqrt(numFeatures).ceil.toInt)
    +    checkFeatureSubsetStrategy(numTrees = 1, "log2",
    +      (math.log(numFeatures) / math.log(2)).ceil.toInt)
    +    checkFeatureSubsetStrategy(numTrees = 1, "onethird", (numFeatures / 3.0).ceil.toInt)
    +
    +    checkFeatureSubsetStrategy(numTrees = 2, "all", numFeatures)
    +    checkFeatureSubsetStrategy(numTrees = 2, "auto", math.sqrt(numFeatures).ceil.toInt)
    +    checkFeatureSubsetStrategy(numTrees = 2, "sqrt", math.sqrt(numFeatures).ceil.toInt)
    +    checkFeatureSubsetStrategy(numTrees = 2, "log2",
    +      (math.log(numFeatures) / math.log(2)).ceil.toInt)
    +    checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt)
    +  }
    +
    +}
    +
    +object RandomForestSuite {
    +
    +  /**
    +   * 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: RandomForestModel,
    +      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(
    --- End diff --
    
    `RandomForest.trainRegressor` is not covered.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55974486
  
    @jkbradley Thanks Joseph. It makes sense.
    
    It looks good upon very rough browsing. Some minor things:
    
    * Would be nice to have support for without-replacement in Bagging (I noticed a TODO comment there already...)
    * Would be nice to have support for manual feature subset size.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943453
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    --- End diff --
    
    `override protected val allStats: Array[Double] =  new Array[Double](allStatsSize)` and move the doc of `_allStats` to here. Scala compiler can handle this automatically.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55964720
  
    Some graphX test failure


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56082554
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20545/consoleFull) for   PR 2435 at commit [`593b13c`](https://github.com/apache/spark/commit/593b13c1af581c11a52ddf4fbf3fe46959a46515).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55972221
  
    Each row is a single (random) dataset.  The 2 different sets of result columns are for 2 different RF implementations:
    * (numTrees): This is from an earlier commit, after implementing RandomForest to train multiple trees at once.  It does not include any code for feature subsampling.
    * (feature subsets): This is from this current PR's code, after implementing feature subsampling.
    These tests were to identify regressions in DecisionTree, so they are training 1 tree with all of the features (i.e., no feature subsampling).
    I have run other tests with numTrees=10 and with sqrt(numFeatures), and those indicate that multi-model training and feature subsets can speed up training for forests.
    
    (I'll update the description with this clarification.)


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17957577
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   * @param nodeOffset  Pre-computed node offset from [[getNodeOffset]].
    +   */
    +  def nodeUpdate(
    +      nodeOffset: Int,
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]].
    +   * For ordered features only.
    +   */
    +  def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = {
    +    require(!isUnordered(featureIndex),
    +      s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" +
    +        s" for unordered feature $featureIndex.")
    +    nodeIndex * nodeStride + featureOffsets(featureIndex)
    +  }
    +
    +  /**
    +   * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]].
    +   * For unordered features only.
    +   */
    +  def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = {
    +    require(isUnordered(featureIndex),
    --- 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17764224
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   * @param nodeOffset  Pre-computed node offset from [[getNodeOffset]].
    +   */
    +  def nodeUpdate(
    +      nodeOffset: Int,
    +      nodeIndex: Int,
    --- End diff --
    
    This is to stay consistent with the abstract class API.  The other instance of this class (for subsampling features) does require nodeIndex.  I agree this is awkward, but I wanted to keep the same API (so that the RF aggregation code does not need to know the difference).


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r19732330
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- End diff --
    
    +1 for `sqrt` as it has been widely used as a standard default for such ensemble methods.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

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


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55975101
  
    @codedeft  For w/o replacement bagging, I definitely agree, and I'll make a JIRA for that after this PR is merged.  For manual feature subset size, what sounds best to you?  Does the suggestion for a user-specified fraction in the "Design decision" section above sound reasonable, or is there a better way?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18008291
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    +
    +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
    +   */
    +  def convertToBaggedRDD[Datum](
    +      input: RDD[Datum],
    +      numSubsamples: Int,
    +      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    --- End diff --
    
    I used DRand, copying SampledRDD, and DRand takes an Int, not a Long.  Should I use a different random number generator?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18010488
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    --- End diff --
    
    I currently have Strategy specific to DecisionTree since I am trying to keep the DecisionTree and RandomForest APIs separate.  I could add it to the private DecisionTreeMetadata class, but this may be the only place this test occurs.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18014923
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    --- End diff --
    
    (See comment below)


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18009800
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala ---
    @@ -128,13 +139,34 @@ private[tree] object DecisionTreeMetadata {
           }
         }
     
    +    // Set number of features to use per node (for random forests).
    +    val _featureSubsetStrategy = featureSubsetStrategy match {
    +      case "auto" => if (numTrees == 1) "all" else "sqrt"
    +      case _ => featureSubsetStrategy
    +    }
    +    val numFeaturesPerNode: Int = _featureSubsetStrategy match {
    +      case "all" => numFeatures
    +      case "sqrt" => math.sqrt(numFeatures).ceil.toInt
    +      case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt)
    --- End diff --
    
    R's randomForest defaults to 1/3 for regression.
    Anecdotally, I've seen problems where a larger number of features work better -- log2/sqrt might be a bit too severe, particularly when you don't have many features to begin with.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55968376
  
    @codedeft  No problem; I apologize for how large the PR is.  I agree this should be merged before further optimizations are made.  This does not include node caching; that will be part of [this JIRA](https://issues.apache.org/jira/browse/SPARK-3161).
    
    With respect to this PR, it would be great if you could check the design decisions listed in the PR description, and let me know if you have suggestions.  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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943429
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    +          featuresForNodes: Option[Map[Int, Map[Int, Array[Int]]]]) =
    +        RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng)
    +      // Sanity check (should never occur):
    +      assert(nodesForGroup.size > 0,
    +        s"RandomForest selected empty nodesForGroup.  Error for unknown reason.")
    +
    +      // Choose node splits, and enqueue new nodes as needed.
    +      timer.start("findBestSplits")
    +      DecisionTree.findBestSplits(baggedInput,
    +        metadata, topNodes, nodesForGroup, featuresForNodes, splits, bins, nodeQueue, timer)
    +      timer.stop("findBestSplits")
    +    }
    +
    +    timer.stop("total")
    +
    +    logInfo("Internal timing for DecisionTree:")
    +    logInfo(s"$timer")
    +
    +    val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo))
    +    RandomForestModel.build(trees)
    +  }
    +
    +}
    +
    +object RandomForest extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Classification,
    +      s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param numClassesForClassification number of classes for classification.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "gini" (recommended) or "entropy".
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Classification, impurityType, maxDepth,
    +      numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo)
    +    trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]]
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainClassifier(input.rdd, numClassesForClassification,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Regression,
    +      s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "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.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Regression, impurityType, maxDepth,
    +      0, maxBins, Sort, categoricalFeaturesInfo)
    +    trainRegressor(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]]
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainRegressor(input.rdd,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * List of supported feature subset sampling strategies.
    +   */
    +  val supportedFeatureSubsetStrategies: Array[String] =
    +    Array("auto", "all", "sqrt", "log2", "onethird")
    +
    +  /**
    +   * Pull nodes off of the queue to split.
    +   * Track memory usage for aggregates, and stop adding nodes when too large.
    +   * @param nodeQueue  Queue of nodes to split.
    +   * @param maxMemoryUsage  Bound on size of aggregate statistics.
    +   * @return  (nodesForGroup, featuresForNodes).
    +   *          nodesForGroup holds the nodes to split: treeIndex --> nodes in tree.
    +   *          featuresForNodes holds selected features for each node:
    +   *            treeIndex --> node index --> feature indices.
    +   *          featuresForNodes is only used when subsampling features;
    +   *          it is empty if not subsampling.
    +   */
    +  private[tree] def selectNodesToSplit(
    +      nodeQueue: mutable.Queue[(Int, Node)],
    +      maxMemoryUsage: Long,
    +      metadata: DecisionTreeMetadata,
    +      rng: scala.util.Random): (Map[Int, Array[Node]], Option[Map[Int, Map[Int, Array[Int]]]]) = {
    +    // Collect some nodes to split:
    +    //  nodesForGroup(treeIndex) = nodes to split
    +    val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]()
    +    val mutableFeaturesForNodes = new mutable.HashMap[Int, mutable.HashMap[Int, Array[Int]]]()
    +    var memUsage: Long = 0L
    +    while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) {
    +      val (treeIndex, node) = nodeQueue.head
    +      // Choose subset of features for node (if subsampling).
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // TODO: Use more efficient subsampling?
    --- End diff --
    
    use selection-and-rejection or reservoir (doesn't need to be in this PR)


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17765183
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute node offset for use with [[nodeUpdate]].
    +   */
    +  def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride
    +
    +  /**
    +   * Faster version of [[update]].
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   * @param nodeOffset  Pre-computed node offset from [[getNodeOffset]].
    +   */
    +  def nodeUpdate(
    +      nodeOffset: Int,
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    +  }
    +
    +  /**
    +   * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]].
    +   * For ordered features only.
    +   */
    +  def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = {
    +    require(!isUnordered(featureIndex),
    +      s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" +
    +        s" for unordered feature $featureIndex.")
    +    nodeIndex * nodeStride + featureOffsets(featureIndex)
    +  }
    +
    +  /**
    +   * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]].
    +   * For unordered features only.
    +   */
    +  def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = {
    +    require(isUnordered(featureIndex),
    --- End diff --
    
    Can we promote this function to the base class `DTStatsAggregator` like this:
    
    ```scala
      def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = {
        val baseOffset = getNodeFeatureOffset(nodeIndex, featureIndex)
        (baseOffset, baseOffset + (metadata.numBins(featureIndex) >> 1) * statsSize)
      }
    ```
    
    This can  avoid duplicate code in derived class.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56792111
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20786/consoleFull) for   PR 2435 at commit [`ef7c293`](https://github.com/apache/spark/commit/ef7c293838e8ffbea2a847f1ef682dfd7a689fb6).
     * This patch **passes** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class RandomForestModel(val trees: Array[DecisionTreeModel], val 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55973870
  
    Jenkins, retest this please.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55957173
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/128/consoleFull) for   PR 2435 at commit [`ff8bb96`](https://github.com/apache/spark/commit/ff8bb967796be47c646bf6ff72eebe960b7d43c7).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943438
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    --- End diff --
    
    insert an empty line to separate imports


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-57095860
  
    @manishamde  Fixed the typo.  I believe I have addressed everything, so please let me know if it looks good.  Thank you for the 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943437
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    +          featuresForNodes: Option[Map[Int, Map[Int, Array[Int]]]]) =
    +        RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng)
    +      // Sanity check (should never occur):
    +      assert(nodesForGroup.size > 0,
    +        s"RandomForest selected empty nodesForGroup.  Error for unknown reason.")
    +
    +      // Choose node splits, and enqueue new nodes as needed.
    +      timer.start("findBestSplits")
    +      DecisionTree.findBestSplits(baggedInput,
    +        metadata, topNodes, nodesForGroup, featuresForNodes, splits, bins, nodeQueue, timer)
    +      timer.stop("findBestSplits")
    +    }
    +
    +    timer.stop("total")
    +
    +    logInfo("Internal timing for DecisionTree:")
    +    logInfo(s"$timer")
    +
    +    val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo))
    +    RandomForestModel.build(trees)
    +  }
    +
    +}
    +
    +object RandomForest extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Classification,
    +      s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param numClassesForClassification number of classes for classification.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "gini" (recommended) or "entropy".
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Classification, impurityType, maxDepth,
    +      numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo)
    +    trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]]
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainClassifier(input.rdd, numClassesForClassification,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Regression,
    +      s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "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.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Regression, impurityType, maxDepth,
    +      0, maxBins, Sort, categoricalFeaturesInfo)
    +    trainRegressor(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]]
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainRegressor(input.rdd,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * List of supported feature subset sampling strategies.
    +   */
    +  val supportedFeatureSubsetStrategies: Array[String] =
    +    Array("auto", "all", "sqrt", "log2", "onethird")
    +
    +  /**
    +   * Pull nodes off of the queue to split.
    +   * Track memory usage for aggregates, and stop adding nodes when too large.
    +   * @param nodeQueue  Queue of nodes to split.
    +   * @param maxMemoryUsage  Bound on size of aggregate statistics.
    +   * @return  (nodesForGroup, featuresForNodes).
    +   *          nodesForGroup holds the nodes to split: treeIndex --> nodes in tree.
    +   *          featuresForNodes holds selected features for each node:
    +   *            treeIndex --> node index --> feature indices.
    +   *          featuresForNodes is only used when subsampling features;
    +   *          it is empty if not subsampling.
    +   */
    +  private[tree] def selectNodesToSplit(
    +      nodeQueue: mutable.Queue[(Int, Node)],
    +      maxMemoryUsage: Long,
    +      metadata: DecisionTreeMetadata,
    +      rng: scala.util.Random): (Map[Int, Array[Node]], Option[Map[Int, Map[Int, Array[Int]]]]) = {
    +    // Collect some nodes to split:
    +    //  nodesForGroup(treeIndex) = nodes to split
    +    val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]()
    +    val mutableFeaturesForNodes = new mutable.HashMap[Int, mutable.HashMap[Int, Array[Int]]]()
    +    var memUsage: Long = 0L
    +    while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) {
    +      val (treeIndex, node) = nodeQueue.head
    +      // Choose subset of features for node (if subsampling).
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // TODO: Use more efficient subsampling?
    +        Some(rng.shuffle(Range(0, metadata.numFeatures).toList)
    +          .take(metadata.numFeaturesPerNode).toArray)
    +      } else {
    +        None
    +      }
    +      val nodeMemUsage = RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +      if (memUsage + nodeMemUsage <= maxMemoryUsage) {
    +        nodeQueue.dequeue()
    +        mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[Node]()) += node
    +        if (featureSubset.nonEmpty) {
    +          mutableFeaturesForNodes
    +            .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, Array[Int]]())(node.id)
    +            = featureSubset.get
    +        }
    +      }
    +      memUsage += nodeMemUsage
    +    }
    +    val nodesForGroup: Map[Int, Array[Node]] = mutableNodesForGroup.mapValues(_.toArray).toMap
    +    val featuresForNodes = if (metadata.subsamplingFeatures) {
    +      Some(mutableFeaturesForNodes.mapValues(_.toMap).toMap)
    +    } else {
    +      None
    +    }
    +    (nodesForGroup, featuresForNodes)
    +  }
    +
    +  /**
    +   * Get the number of values to be stored for this node in the bin aggregates.
    +   * @param featureSubset  Indices of features which may be split at this node.
    +   *                       If None, then use all features.
    +   */
    +  private[tree] def numElementsForNode(
    +      metadata: DecisionTreeMetadata,
    +      featureSubset: Option[Array[Int]]): Long = {
    --- End diff --
    
    Should `Int` be sufficient?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18051027
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,429 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    // TODO: Calculate memory usage more precisely.
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    --- End diff --
    
    Need to clarify that this is done across multiple trees. If I understand correctly, the two maps returned are indexed by the tree id. Also, a little more documentation in the selectNodesToSplit might be helpful for future code maintenance.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18008327
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    +          featuresForNodes: Option[Map[Int, Map[Int, Array[Int]]]]) =
    +        RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng)
    +      // Sanity check (should never occur):
    +      assert(nodesForGroup.size > 0,
    +        s"RandomForest selected empty nodesForGroup.  Error for unknown reason.")
    +
    +      // Choose node splits, and enqueue new nodes as needed.
    +      timer.start("findBestSplits")
    +      DecisionTree.findBestSplits(baggedInput,
    +        metadata, topNodes, nodesForGroup, featuresForNodes, splits, bins, nodeQueue, timer)
    +      timer.stop("findBestSplits")
    +    }
    +
    +    timer.stop("total")
    +
    +    logInfo("Internal timing for DecisionTree:")
    +    logInfo(s"$timer")
    +
    +    val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo))
    +    RandomForestModel.build(trees)
    +  }
    +
    +}
    +
    +object RandomForest extends Serializable with Logging {
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Classification,
    +      s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for binary or multiclass classification.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels should take values {0, 1, ..., numClasses-1}.
    +   * @param numClassesForClassification number of classes for classification.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "gini" (recommended) or "entropy".
    +   * @param maxDepth Maximum depth of the tree.
    +   *                 E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainClassifier(
    +      input: RDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Classification, impurityType, maxDepth,
    +      numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo)
    +    trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]]
    +   */
    +  def trainClassifier(
    +      input: JavaRDD[LabeledPoint],
    +      numClassesForClassification: Int,
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainClassifier(input.rdd, numClassesForClassification,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param strategy Parameters for training each tree in the forest.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      strategy: Strategy,
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      seed: Int): RandomForestModel = {
    +    require(strategy.algo == Regression,
    +      s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}")
    +    val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed)
    +    rf.train(input)
    +  }
    +
    +  /**
    +   * Method to train a decision tree model for regression.
    +   *
    +   * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
    +   *              Labels are real numbers.
    +   * @param categoricalFeaturesInfo Map storing arity of categorical features.
    +   *                                E.g., an entry (n -> k) indicates that feature n is categorical
    +   *                                with k categories indexed from 0: {0, 1, ..., k-1}.
    +   * @param numTrees Number of trees in the random forest.
    +   * @param featureSubsetStrategy Number of features to consider for splits at each node.
    +   *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    +   *                              If "auto" is set, this parameter is set based on numTrees:
    +   *                              if numTrees == 1, then featureSubsetStrategy = "all";
    +   *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    +   * @param impurity Criterion used for information gain calculation.
    +   *                 Supported values: "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.
    +   *                  (suggested value: 4)
    +   * @param maxBins maximum number of bins used for splitting features
    +   *                 (suggested value: 100)
    +   * @param seed  Random seed for bootstrapping and choosing feature subsets.
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def trainRegressor(
    +      input: RDD[LabeledPoint],
    +      categoricalFeaturesInfo: Map[Int, Int],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int = Utils.random.nextInt()): RandomForestModel = {
    +    val impurityType = Impurities.fromString(impurity)
    +    val strategy = new Strategy(Regression, impurityType, maxDepth,
    +      0, maxBins, Sort, categoricalFeaturesInfo)
    +    trainRegressor(input, strategy, numTrees, featureSubsetStrategy, seed)
    +  }
    +
    +  /**
    +   * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]]
    +   */
    +  def trainRegressor(
    +      input: JavaRDD[LabeledPoint],
    +      categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
    +      numTrees: Int,
    +      featureSubsetStrategy: String,
    +      impurity: String,
    +      maxDepth: Int,
    +      maxBins: Int,
    +      seed: Int): RandomForestModel = {
    +    trainRegressor(input.rdd,
    +      categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
    +      numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed)
    +  }
    +
    +  /**
    +   * List of supported feature subset sampling strategies.
    +   */
    +  val supportedFeatureSubsetStrategies: Array[String] =
    +    Array("auto", "all", "sqrt", "log2", "onethird")
    +
    +  /**
    +   * Pull nodes off of the queue to split.
    +   * Track memory usage for aggregates, and stop adding nodes when too large.
    +   * @param nodeQueue  Queue of nodes to split.
    +   * @param maxMemoryUsage  Bound on size of aggregate statistics.
    +   * @return  (nodesForGroup, featuresForNodes).
    +   *          nodesForGroup holds the nodes to split: treeIndex --> nodes in tree.
    +   *          featuresForNodes holds selected features for each node:
    +   *            treeIndex --> node index --> feature indices.
    +   *          featuresForNodes is only used when subsampling features;
    +   *          it is empty if not subsampling.
    +   */
    +  private[tree] def selectNodesToSplit(
    +      nodeQueue: mutable.Queue[(Int, Node)],
    +      maxMemoryUsage: Long,
    +      metadata: DecisionTreeMetadata,
    +      rng: scala.util.Random): (Map[Int, Array[Node]], Option[Map[Int, Map[Int, Array[Int]]]]) = {
    +    // Collect some nodes to split:
    +    //  nodesForGroup(treeIndex) = nodes to split
    +    val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]()
    +    val mutableFeaturesForNodes = new mutable.HashMap[Int, mutable.HashMap[Int, Array[Int]]]()
    +    var memUsage: Long = 0L
    +    while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) {
    +      val (treeIndex, node) = nodeQueue.head
    +      // Choose subset of features for node (if subsampling).
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // TODO: Use more efficient subsampling?
    +        Some(rng.shuffle(Range(0, metadata.numFeatures).toList)
    +          .take(metadata.numFeaturesPerNode).toArray)
    +      } else {
    +        None
    +      }
    +      val nodeMemUsage = RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +      if (memUsage + nodeMemUsage <= maxMemoryUsage) {
    +        nodeQueue.dequeue()
    +        mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[Node]()) += node
    +        if (featureSubset.nonEmpty) {
    +          mutableFeaturesForNodes
    +            .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, Array[Int]]())(node.id)
    +            = featureSubset.get
    +        }
    +      }
    +      memUsage += nodeMemUsage
    +    }
    +    val nodesForGroup: Map[Int, Array[Node]] = mutableNodesForGroup.mapValues(_.toArray).toMap
    +    val featuresForNodes = if (metadata.subsamplingFeatures) {
    +      Some(mutableFeaturesForNodes.mapValues(_.toMap).toMap)
    +    } else {
    +      None
    +    }
    +    (nodesForGroup, featuresForNodes)
    +  }
    +
    +  /**
    +   * Get the number of values to be stored for this node in the bin aggregates.
    +   * @param featureSubset  Indices of features which may be split at this node.
    +   *                       If None, then use all features.
    +   */
    +  private[tree] def numElementsForNode(
    +      metadata: DecisionTreeMetadata,
    +      featureSubset: Option[Array[Int]]): Long = {
    --- End diff --
    
    Probably, but it would be possible to get up to max Int and still fit in memory (training perhaps 1 node per iteration).


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18063715
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -465,43 +329,60 @@ object DecisionTree extends Serializable with Logging {
        * @param agg  Array storing aggregate calculation, with a set of sufficient statistics for
        *             each (node, feature, bin).
        * @param treePoint  Data point being aggregated.
    -   * @param nodeIndex  Node corresponding to treePoint. Indexed from 0 at start of (level, group).
    +   * @param nodeIndex  Node corresponding to treePoint.  agg is indexed in [0, numNodes).
        * @param bins possible bins for all features, indexed (numFeatures)(numBins)
        * @param unorderedFeatures  Set of indices of unordered features.
    +   * @param instanceWeight  Weight (importance) of instance in dataset.
        */
       private def mixedBinSeqOp(
           agg: DTStatsAggregator,
           treePoint: TreePoint,
           nodeIndex: Int,
           bins: Array[Array[Bin]],
    -      unorderedFeatures: Set[Int]): Unit = {
    -    // Iterate over all features.
    -    val numFeatures = treePoint.binnedFeatures.size
    +      unorderedFeatures: Set[Int],
    +      instanceWeight: Double,
    +      featuresForNode: Option[Array[Int]]): Unit = {
    +    val numFeaturesPerNode = if (featuresForNode.nonEmpty) {
    +      // Use subsampled features
    +      featuresForNode.get.size
    +    } else {
    +      // Use all features
    +      agg.metadata.numFeatures
    +    }
         val nodeOffset = agg.getNodeOffset(nodeIndex)
    -    var featureIndex = 0
    -    while (featureIndex < numFeatures) {
    +    // Iterate over features.
    +    var featureIndexIdx = 0
    +    while (featureIndexIdx < numFeaturesPerNode) {
    +      val featureIndex = if (featuresForNode.nonEmpty) {
    +        featuresForNode.get.apply(featureIndexIdx)
    --- End diff --
    
    minor: apply is a bit awkward since you normally use the syntactic sugar () to access array elements.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-61482956
  
    Any ideas or plans to support more types of ensemble of decision tree (such as Extra-Trees, Bagging)? I'd be more than happy to help with those... where should I start?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18010171
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    --- End diff --
    
    Instead of checking numTrees > 1, it might be better to add a ```def isRandomForest```.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18066800
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -626,21 +492,27 @@ object DecisionTree extends Serializable with Logging {
          *
          * @param agg  Array storing aggregate calculation, with a set of sufficient statistics for
          *             each (node, feature, bin).
    -     * @param treePoint   Data point being aggregated.
    +     * @param baggedPoint   Data point being aggregated.
          * @return  agg
          */
         def binSeqOp(
             agg: DTStatsAggregator,
    -        treePoint: TreePoint): DTStatsAggregator = {
    -      val nodeIndex = treePointToNodeIndex(treePoint)
    -      // If the example does not reach this level, then nodeIndex < 0.
    -      // If the example reaches this level but is handled in a different group,
    -      //  then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group).
    -      if (nodeIndex >= 0 && nodeIndex < numNodes) {
    -        if (metadata.unorderedFeatures.isEmpty) {
    -          orderedBinSeqOp(agg, treePoint, nodeIndex)
    -        } else {
    -          mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures)
    +        baggedPoint: BaggedPoint[TreePoint]): DTStatsAggregator = {
    +      treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) =>
    +        val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures,
    +          bins, metadata.unorderedFeatures)
    +        val nodeInfo = nodeIndexToInfo.getOrElse(nodeIndex, null)
    +        // If the example does not reach a node in this group, then nodeIndo = null.
    --- End diff --
    
    typo: nodeIndo


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18010322
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    +        Some(metadata.numBins.zipWithIndex.sortBy(- _._1)
    +          .take(metadata.numFeaturesPerNode).map(_._2))
    +      } else {
    +        None
    +      }
    +      RandomForest.numElementsForNode(metadata, featureSubset) * 8L
    +    }
    +    require(maxMemoryPerNode <= maxMemoryUsage,
    +      s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," +
    +      " which is too small for the given features." +
    +      s"  Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}")
    +    // TODO: Calculate memory usage more precisely.
    +
    +    timer.stop("init")
    +
    +    /*
    +     * The main idea here is to perform group-wise training of the decision tree nodes thus
    +     * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup).
    +     * Each data sample is handled by a particular node (or it reaches a leaf and is not used
    +     * in lower levels).
    +     */
    +
    +    // FIFO queue of nodes to train: (treeIndex, node)
    +    val nodeQueue = new mutable.Queue[(Int, Node)]()
    +
    +    val rng = new scala.util.Random()
    +    rng.setSeed(seed)
    +
    +    // Allocate and queue root nodes.
    +    val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1))
    +    Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex))))
    +
    +    while (nodeQueue.nonEmpty) {
    +      // Collect some nodes to split, and choose features for each node (if subsampling).
    +      val (nodesForGroup: Map[Int, Array[Node]],
    --- End diff --
    
    Cool.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56764685
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20774/consoleFull) for   PR 2435 at commit [`ef7c293`](https://github.com/apache/spark/commit/ef7c293838e8ffbea2a847f1ef682dfd7a689fb6).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class RandomForestModel(val trees: Array[DecisionTreeModel], val 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: [SPARK-1545] [mllib] Add Random Forests

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

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


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55975519
  
    @jkbradley I guess that I don't have a particular preference, (either fraction or the actual number). The actual number seems a bit better to me since you are not going to get confused by rounding up/down. R's randomForest, H2O, wise.io all seem to like the actual number, and they all call it 'mtry'.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17957573
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    --- 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943493
  
    --- Diff: mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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.mutable
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.linalg.Vectors
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata}
    +import org.apache.spark.mllib.tree.impurity.Gini
    +import org.apache.spark.mllib.tree.model.{Node, RandomForestModel}
    +import org.apache.spark.mllib.util.LocalSparkContext
    +import org.apache.spark.util.StatCounter
    +
    +/**
    + * Test suite for [[RandomForest]].
    + */
    +class RandomForestSuite extends FunSuite with LocalSparkContext {
    +
    +  test("BaggedPoint RDD: without subsampling") {
    +    val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1)
    +    val rdd = sc.parallelize(arr)
    +    val baggedRDD = BaggedPoint.convertToBaggedRDDWithoutSampling(rdd)
    +    baggedRDD.collect().foreach { baggedPoint =>
    +      assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1)
    +    }
    +  }
    +
    +  test("BaggedPoint RDD: with subsampling") {
    +    val numSubsamples = 100
    +    val (expectedMean, expectedStddev) = (1.0, 1.0)
    +
    +    val seeds = Array(123, 5354, 230, 349867, 23987)
    +    val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1)
    +    val rdd = sc.parallelize(arr)
    +    seeds.foreach { seed =>
    +      val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, numSubsamples, seed = seed)
    +      val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect()
    +      RandomForestSuite.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev,
    --- End diff --
    
    line too wide


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55976071
  
    Additionally, I suppose allowing the actual size for feature subset as an input would be useful in model-search later on.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56761597
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20774/consoleFull) for   PR 2435 at commit [`ef7c293`](https://github.com/apache/spark/commit/ef7c293838e8ffbea2a847f1ef682dfd7a689fb6).
     * 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18018595
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    +
    +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
    +   */
    +  def convertToBaggedRDD[Datum](
    +      input: RDD[Datum],
    +      numSubsamples: Int,
    +      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    --- End diff --
    
    Use `seed.toInt` for `DRand`.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18053368
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    --- End diff --
    
    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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17957595
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    +
    +  /**
    +   * Flat array of elements.
    +   * Index for start of stats for a (node, feature, bin) is:
    +   *   index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +   * Note: For unordered features, the left child stats precede the right child stats
    +   *       in the binIndex order.
    +   */
    +  protected val _allStats: Array[Double] = new Array[Double](allStatsSize)
    +
    +  /**
    +   * Get flat array of elements stored in this aggregator.
    +   */
    +  protected def allStats: Array[Double] = _allStats
    +
    +  /**
    +   * Update the stats for a given (node, feature, bin) for ordered features, using the given label.
    +   */
    +  def update(
    +      nodeIndex: Int,
    +      featureIndex: Int,
    +      binIndex: Int,
    +      label: Double,
    +      instanceWeight: Double): Unit = {
    +    val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize
    +    impurityAggregator.update(_allStats, i, label, instanceWeight)
    --- End diff --
    
    I'll see about improving this, though I may go with just using allStats everywhere.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943451
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala ---
    @@ -189,6 +160,230 @@ private[tree] class DTStatsAggregator(
         }
         this
       }
    +}
    +
    +/**
    + * DecisionTree statistics aggregator.
    + * This holds a flat array of statistics for a set of (nodes, features, bins)
    + * and helps with indexing.
    + *
    + * This instance of [[DTStatsAggregator]] is used when not subsampling features.
    + *
    + * @param numNodes  Number of nodes to collect statistics for.
    + */
    +private[tree] class DTStatsAggregatorFixedFeatures(
    +    metadata: DecisionTreeMetadata,
    +    numNodes: Int) extends DTStatsAggregator(metadata) {
    +
    +  /**
    +   * Offset for each feature for calculating indices into the [[_allStats]] array.
    +   * Mapping: featureIndex --> offset
    +   */
    +  private val featureOffsets: Array[Int] = {
    +    metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins)
    +  }
    +
    +  /**
    +   * Number of elements for each node, corresponding to stride between nodes in [[_allStats]].
    +   */
    +  private val nodeStride: Int = featureOffsets.last
    +
    +  /**
    +   * Total number of elements stored in this aggregator.
    +   */
    +  def allStatsSize: Int = numNodes * nodeStride
    --- End diff --
    
    `def` -> `override val` and remove doc.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-55971575
  
    @jkbradley I don't quite get what different columns in result numbers mean.
    
    Do you mean that you are still training exactly the same single tree (to depth 6) on different number of rows/features but with added features?
    
    Are you still sampling 100% of features per node even with feature subsets in these tests?



---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-57116574
  
    LGTM and tested with 1000 trees. I've merged it into master. Thanks @jkbradley for implementing RF and @codedeft @manishamde @chouqin for reviewing!


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-61526469
  
    Thanks @manishamde !
     
    Found no JIRA related to the Extra-Trees, I created the following ticket: https://issues.apache.org/jira/browse/SPARK-4210? Is it okay?


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56785203
  
    test this please


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#issuecomment-56904403
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20827/consoleFull) for   PR 2435 at commit [`cc59d78`](https://github.com/apache/spark/commit/cc59d78bc45f5c0b8a0597121c89f932912e2bb3).
     * This patch **passes** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class RandomForestModel(val trees: Array[DecisionTreeModel], val 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17943404
  
    --- Diff: examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala ---
    @@ -228,4 +253,23 @@ object DecisionTreeRunner {
           err * err
         }.mean()
       }
    +
    +  /**
    +   * Calculates the classifier accuracy.
    +   */
    +  private def accuracyScore(model: RandomForestModel, data: RDD[LabeledPoint]): Double = {
    --- End diff --
    
    minor: we can use `MulticlassMetrics` 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18066648
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala ---
    @@ -465,43 +329,60 @@ object DecisionTree extends Serializable with Logging {
        * @param agg  Array storing aggregate calculation, with a set of sufficient statistics for
        *             each (node, feature, bin).
        * @param treePoint  Data point being aggregated.
    -   * @param nodeIndex  Node corresponding to treePoint. Indexed from 0 at start of (level, group).
    +   * @param nodeIndex  Node corresponding to treePoint.  agg is indexed in [0, numNodes).
        * @param bins possible bins for all features, indexed (numFeatures)(numBins)
        * @param unorderedFeatures  Set of indices of unordered features.
    +   * @param instanceWeight  Weight (importance) of instance in dataset.
        */
       private def mixedBinSeqOp(
           agg: DTStatsAggregator,
           treePoint: TreePoint,
           nodeIndex: Int,
           bins: Array[Array[Bin]],
    -      unorderedFeatures: Set[Int]): Unit = {
    -    // Iterate over all features.
    -    val numFeatures = treePoint.binnedFeatures.size
    +      unorderedFeatures: Set[Int],
    +      instanceWeight: Double,
    +      featuresForNode: Option[Array[Int]]): Unit = {
    +    val numFeaturesPerNode = if (featuresForNode.nonEmpty) {
    +      // Use subsampled features
    +      featuresForNode.get.size
    +    } else {
    +      // Use all features
    +      agg.metadata.numFeatures
    +    }
         val nodeOffset = agg.getNodeOffset(nodeIndex)
    -    var featureIndex = 0
    -    while (featureIndex < numFeatures) {
    +    // Iterate over features.
    +    var featureIndexIdx = 0
    +    while (featureIndexIdx < numFeaturesPerNode) {
    +      val featureIndex = if (featuresForNode.nonEmpty) {
    +        featuresForNode.get.apply(featureIndexIdx)
    --- End diff --
    
    I agree. We could also split it into two lines. :-) Let's keep it as it is.


---
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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r18053331
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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 cern.jet.random.Poisson
    +import cern.jet.random.engine.DRand
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Internal representation of a datapoint which belongs to several subsamples of the same dataset,
    + * particularly for bagging (e.g., for random forests).
    + *
    + * This holds one instance, as well as an array of weights which represent the (weighted)
    + * number of times which this instance appears in each subsample.
    + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that
    + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively.
    + *
    + * @param datum  Data instance
    + * @param subsampleWeights  Weight of this instance in each subsampled dataset.
    + *
    + * TODO: This does not currently support (Double) weighted instances.  Once MLlib has weighted
    + *       dataset support, update.  (We store subsampleWeights as Double for this future extension.)
    + */
    +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double])
    +  extends Serializable {
    +}
    +
    +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
    +   */
    +  def convertToBaggedRDD[Datum](
    +      input: RDD[Datum],
    +      numSubsamples: Int,
    +      seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = {
    --- End diff --
    
    OK, but that seems a little deceptive in terms of 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: [SPARK-1545] [mllib] Add Random Forests

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

    https://github.com/apache/spark/pull/2435#discussion_r17889641
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala ---
    @@ -0,0 +1,430 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.annotation.Experimental
    +import org.apache.spark.api.java.JavaRDD
    +import org.apache.spark.mllib.regression.LabeledPoint
    +import org.apache.spark.mllib.tree.configuration.Algo._
    +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
    +import org.apache.spark.mllib.tree.configuration.Strategy
    +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker}
    +import org.apache.spark.mllib.tree.impurity.Impurities
    +import org.apache.spark.mllib.tree.model._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: Experimental ::
    + * A class which implements a random forest learning algorithm for classification and regression.
    + * It supports both continuous and categorical features.
    + *
    + * @param strategy The configuration parameters for the random forest algorithm which specify
    + *                 the type of algorithm (classification, regression, etc.), feature type
    + *                 (continuous, categorical), depth of the tree, quantile calculation strategy,
    + *                 etc.
    + * @param numTrees If 1, then no bootstrapping is used.  If > 1, then bootstrapping is done.
    + * @param featureSubsetStrategy Number of features to consider for splits at each node.
    + *                              Supported: "auto" (default), "all", "sqrt", "log2", "onethird".
    + *                              If "auto" is set, this parameter is set based on numTrees:
    + *                              if numTrees == 1, then featureSubsetStrategy = "all";
    + *                              if numTrees > 1, then featureSubsetStrategy = "sqrt".
    + * @param seed  Random seed for bootstrapping and choosing feature subsets.
    + */
    +@Experimental
    +private class RandomForest (
    +    private val strategy: Strategy,
    +    private val numTrees: Int,
    +    featureSubsetStrategy: String,
    +    private val seed: Int)
    +  extends Serializable with Logging {
    +
    +  strategy.assertValid()
    +  require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.")
    +  require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy),
    +    s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." +
    +    s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.")
    +
    +  /**
    +   * Method to train a decision tree model over an RDD
    +   * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]
    +   * @return RandomForestModel that can be used for prediction
    +   */
    +  def train(input: RDD[LabeledPoint]): RandomForestModel = {
    +
    +    val timer = new TimeTracker()
    +
    +    timer.start("total")
    +
    +    timer.start("init")
    +
    +    val retaggedInput = input.retag(classOf[LabeledPoint])
    +    val metadata =
    +      DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy)
    +    logDebug("algo = " + strategy.algo)
    +    logDebug("numTrees = " + numTrees)
    +    logDebug("seed = " + seed)
    +    logDebug("maxBins = " + metadata.maxBins)
    +    logDebug("featureSubsetStrategy = " + featureSubsetStrategy)
    +    logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode)
    +
    +    // Find the splits and the corresponding bins (interval between the splits) using a sample
    +    // of the input data.
    +    timer.start("findSplitsBins")
    +    val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata)
    +    timer.stop("findSplitsBins")
    +    logDebug("numBins: feature: number of bins")
    +    logDebug(Range(0, metadata.numFeatures).map { featureIndex =>
    +        s"\t$featureIndex\t${metadata.numBins(featureIndex)}"
    +      }.mkString("\n"))
    +
    +    // Bin feature values (TreePoint representation).
    +    // Cache input RDD for speedup during multiple passes.
    +    val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata)
    +    val baggedInput = if (numTrees > 1) {
    +      BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed)
    +    } else {
    +      BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput)
    +    }.persist(StorageLevel.MEMORY_AND_DISK)
    +
    +    // depth of the decision tree
    +    val maxDepth = strategy.maxDepth
    +    require(maxDepth <= 30,
    +      s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.")
    +
    +    // Max memory usage for aggregates
    +    val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L
    +    logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.")
    +    val maxMemoryPerNode = {
    +      val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) {
    +        // Find numFeaturesPerNode largest bins to get an upper bound on memory usage.
    --- End diff --
    
    Should we add a TODO and JIRA to improve this estimate?


---
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