Skip to content

Commit

Permalink
DecisionTree API change:
Browse files Browse the repository at this point in the history
* Added 6 static train methods to match Python API, but without default arguments (but with Python default args noted in docs).

Added factory classes for Algo and Impurity, but made private[mllib].
  • Loading branch information
jkbradley committed Aug 6, 2014
1 parent c699850 commit e358661
Show file tree
Hide file tree
Showing 4 changed files with 235 additions and 48 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,9 @@ import org.apache.spark.mllib.random.{RandomRDDGenerators => RG}
import org.apache.spark.mllib.recommendation._
import org.apache.spark.mllib.regression._
import org.apache.spark.mllib.tree.configuration.Algo._
import org.apache.spark.mllib.tree.configuration.Strategy
import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
import org.apache.spark.mllib.tree.DecisionTree
import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance}
import org.apache.spark.mllib.tree.impurity._
import org.apache.spark.mllib.tree.model.DecisionTreeModel
import org.apache.spark.mllib.stat.Statistics
import org.apache.spark.mllib.stat.correlation.CorrelationNames
Expand Down Expand Up @@ -498,17 +498,8 @@ class PythonMLLibAPI extends Serializable {

val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint)

val algo: Algo = algoStr match {
case "classification" => Classification
case "regression" => Regression
case _ => throw new IllegalArgumentException(s"Bad algoStr parameter: $algoStr")
}
val impurity: Impurity = impurityStr match {
case "gini" => Gini
case "entropy" => Entropy
case "variance" => Variance
case _ => throw new IllegalArgumentException(s"Bad impurityStr parameter: $impurityStr")
}
val algo = Algo.stringToAlgo(algoStr)
val impurity = Impurities.stringToImpurity(impurityStr)

val strategy = new Strategy(
algo = algo,
Expand Down
228 changes: 193 additions & 35 deletions mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,16 @@

package org.apache.spark.mllib.tree

import scala.collection.JavaConverters._

import org.apache.spark.annotation.Experimental
import org.apache.spark.Logging
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.tree.configuration.Strategy
import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
import org.apache.spark.mllib.tree.configuration.Algo._
import org.apache.spark.mllib.tree.configuration.FeatureType._
import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
import org.apache.spark.mllib.tree.impurity.Impurity
import org.apache.spark.mllib.tree.impurity.{Impurities, Gini, Entropy, Impurity}
import org.apache.spark.mllib.tree.model._
import org.apache.spark.rdd.RDD
import org.apache.spark.util.random.XORShiftRandom
Expand Down Expand Up @@ -213,10 +215,8 @@ object DecisionTree extends Serializable with Logging {
}

/**
* Method to train a decision tree model where the instances are represented as an RDD of
* (label, features) pairs. The method supports binary classification and regression. For the
* binary classification, the label for each instance should either be 0 or 1 to denote the two
* classes.
* Method to train a decision tree model.
* The method supports binary and multiclass classification and regression.
*
* @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
* For classification, labels should take values {0, 1, ..., numClasses-1}.
Expand All @@ -237,10 +237,8 @@ object DecisionTree extends Serializable with Logging {
}

/**
* Method to train a decision tree model where the instances are represented as an RDD of
* (label, features) pairs. The method supports binary classification and regression. For the
* binary classification, the label for each instance should either be 0 or 1 to denote the two
* classes.
* Method to train a decision tree model.
* The method supports binary and multiclass classification and regression.
*
* @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
* For classification, labels should take values {0, 1, ..., numClasses-1}.
Expand All @@ -263,11 +261,8 @@ object DecisionTree extends Serializable with Logging {
}

/**
* Method to train a decision tree model where the instances are represented as an RDD of
* (label, features) pairs. The decision tree method supports binary classification and
* regression. For the binary classification, the label for each instance should either be 0 or
* 1 to denote the two classes. The method also supports categorical features inputs where the
* number of categories can specified using the categoricalFeaturesInfo option.
* Method to train a decision tree model.
* The method supports binary and multiclass classification and regression.
*
* @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
* For classification, labels should take values {0, 1, ..., numClasses-1}.
Expand All @@ -279,11 +274,9 @@ object DecisionTree extends Serializable with Logging {
* @param numClassesForClassification number of classes for classification. Default value of 2.
* @param maxBins maximum number of bins used for splitting features
* @param quantileCalculationStrategy algorithm for calculating quantiles
* @param categoricalFeaturesInfo A map storing information about the categorical variables and
* the number of discrete values they take. For example,
* an entry (n -> k) implies the feature n is categorical with k
* categories 0, 1, 2, ... , k-1. It's important to note that
* features are zero-indexed.
* @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}.
* @return DecisionTreeModel that can be used for prediction
*/
def train(
Expand All @@ -300,32 +293,197 @@ object DecisionTree extends Serializable with Logging {
new DecisionTree(strategy).train(input)
}

// Optional arguments in Python: maxBins
/**
* Method to train a decision tree model.
* The method supports binary and multiclass classification and regression.
* This version takes basic types, for consistency with Python API.
*
* @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
* For classification, labels should take values {0, 1, ..., numClasses-1}.
* For regression, labels are real numbers.
* @param algo "classification" or "regression"
* @param numClassesForClassification number of classes for classification. Default value of 2.
* @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 impurity criterion used for information gain calculation
* @param maxDepth Maximum depth of the tree.
* E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
* @param maxBins maximum number of bins used for splitting features
* (default Python value = 100)
* @return DecisionTreeModel that can be used for prediction
*/
def train(
input: RDD[LabeledPoint],
algo: Algo,
algo: String,
numClassesForClassification: Int,
categoricalFeaturesInfo: Map[Int,Int],
impurity: Impurity,
categoricalFeaturesInfo: Map[Int, Int],
impurity: String,
maxDepth: Int,
maxBins: Int): DecisionTreeModel = ???
maxBins: Int): DecisionTreeModel = {
val algoType = Algo.stringToAlgo(algo)
val impurityType = Impurities.stringToImpurity(impurity)
train(input, algoType, impurityType, maxDepth, numClassesForClassification, maxBins, Sort,
categoricalFeaturesInfo)
}

// Optional arguments in Python: all but input, numClassesForClassification
/**
* Method to train a decision tree model.
* The method supports binary and multiclass classification and regression.
* This version takes basic types, for consistency with Python API.
* This version is Java-friendly, taking a Java map for categoricalFeaturesInfo.
*
* @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
* For classification, labels should take values {0, 1, ..., numClasses-1}.
* For regression, labels are real numbers.
* @param algo "classification" or "regression"
* @param numClassesForClassification number of classes for classification. Default value of 2.
* @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 impurity criterion used for information gain calculation
* @param maxDepth Maximum depth of the tree.
* E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
* @param maxBins maximum number of bins used for splitting features
* (default Python value = 100)
* @return DecisionTreeModel that can be used for prediction
*/
def train(
input: RDD[LabeledPoint],
algo: String,
numClassesForClassification: Int,
categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
impurity: String,
maxDepth: Int,
maxBins: Int): DecisionTreeModel = {
train(input, algo, numClassesForClassification,
categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
impurity, maxDepth, maxBins)
}

/**
* Method to train a decision tree model for binary or multiclass classification.
* This version takes basic types, for consistency with Python API.
*
* @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}.
* (default Python value = {}, i.e., no categorical features)
* @param impurity criterion used for information gain calculation
* (default Python value = "gini")
* @param maxDepth Maximum depth of the tree.
* E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
* (default Python value = 4)
* @param maxBins maximum number of bins used for splitting features
* (default Python value = 100)
* @return DecisionTreeModel that can be used for prediction
*/
def trainClassifier(
input: RDD[LabeledPoint],
numClassesForClassification: Int,
categoricalFeaturesInfo: Map[Int,Int],
impurity: Impurity,
categoricalFeaturesInfo: Map[Int, Int],
impurity: String,
maxDepth: Int,
maxBins: Int): DecisionTreeModel = {
train(input, "classification", numClassesForClassification, categoricalFeaturesInfo, impurity,
maxDepth, maxBins)
}

/**
* Method to train a decision tree model for binary or multiclass classification.
* This version takes basic types, for consistency with Python API.
* This version is Java-friendly, taking a Java map for categoricalFeaturesInfo.
*
* @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}.
* (default Python value = {}, i.e., no categorical features)
* @param impurity criterion used for information gain calculation
* (default Python value = "gini")
* @param maxDepth Maximum depth of the tree.
* E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.
* (default Python value = 4)
* @param maxBins maximum number of bins used for splitting features
* (default Python value = 100)
* @return DecisionTreeModel that can be used for prediction
*/
def trainClassifier(
input: RDD[LabeledPoint],
numClassesForClassification: Int,
categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
impurity: String,
maxDepth: Int,
maxBins: Int): DecisionTreeModel = {
trainClassifier(input, numClassesForClassification,
categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
impurity, maxDepth, maxBins)
}

/**
* Method to train a decision tree model for regression.
* This version takes basic types, for consistency with Python API.
*
* @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}.
* (default Python value = {}, i.e., no categorical features)
* @param impurity criterion used for information gain calculation
* (default Python value = "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.
* (default Python value = 4)
* @param maxBins maximum number of bins used for splitting features
* (default Python value = 100)
* @return DecisionTreeModel that can be used for prediction
*/
def trainRegressor(
input: RDD[LabeledPoint],
categoricalFeaturesInfo: Map[Int, Int],
impurity: String,
maxDepth: Int,
maxBins: Int): DecisionTreeModel = ???
maxBins: Int): DecisionTreeModel = {
train(input, "regression", 0, categoricalFeaturesInfo, impurity, maxDepth, maxBins)
}

// Optional arguments in Python: all but input
/**
* Method to train a decision tree model for regression.
* This version takes basic types, for consistency with Python API.
* This version is Java-friendly, taking a Java map for categoricalFeaturesInfo.
*
* @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}.
* (default Python value = {}, i.e., no categorical features)
* @param impurity criterion used for information gain calculation
* (default Python value = "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.
* (default Python value = 4)
* @param maxBins maximum number of bins used for splitting features
* (default Python value = 100)
* @return DecisionTreeModel that can be used for prediction
*/
def trainRegressor(
input: RDD[LabeledPoint],
categoricalFeaturesInfo: Map[Int,Int],
impurity: Impurity,
categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer],
impurity: String,
maxDepth: Int,
maxBins: Int): DecisionTreeModel = ???
maxBins: Int): DecisionTreeModel = {
trainRegressor(input,
categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap,
impurity, maxDepth, maxBins)
}


private val InvalidBinIndex = -1

Expand Down Expand Up @@ -1361,10 +1519,10 @@ object DecisionTree extends Serializable with Logging {
* (a) For multiclass classification with a low-arity feature
* (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits),
* the feature is split based on subsets of categories.
* There are 2^(maxFeatureValue - 1) - 1 splits.
* There are math.pow(2, (maxFeatureValue - 1) - 1) splits.
* (b) For regression and binary classification,
* and for multiclass classification with a high-arity feature,
* there is one split per category.
*
* Categorical case (a) features are called unordered features.
* Other cases are called ordered features.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,4 +27,10 @@ import org.apache.spark.annotation.Experimental
object Algo extends Enumeration {
type Algo = Value
val Classification, Regression = Value

private[mllib] def stringToAlgo(name: String): Algo = name match {
case "classification" => Classification
case "regression" => Regression
case _ => throw new IllegalArgumentException(s"Did not recognize Algo name: $name")
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.impurity

/**
* Factory class for Impurity types.
*/
private[mllib] object Impurities {

def stringToImpurity(name: String): Impurity = name match {
case "gini" => Gini
case "entropy" => Entropy
case "variance" => Variance
case _ => throw new IllegalArgumentException(s"Did not recognize Impurity name: $name")
}

}

0 comments on commit e358661

Please sign in to comment.