Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 7 additions & 2 deletions R/pkg/NAMESPACE
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,8 @@ exportMethods("glm",
"spark.als",
"spark.kstest",
"spark.logit",
"spark.randomForest")
"spark.randomForest",
"spark.gbt")

# Job group lifecycle management methods
export("setJobGroup",
Expand Down Expand Up @@ -353,7 +354,9 @@ export("as.DataFrame",
"read.ml",
"print.summary.KSTest",
"print.summary.RandomForestRegressionModel",
"print.summary.RandomForestClassificationModel")
"print.summary.RandomForestClassificationModel",
"print.summary.GBTRegressionModel",
"print.summary.GBTClassificationModel")

export("structField",
"structField.jobj",
Expand All @@ -380,6 +383,8 @@ S3method(print, summary.GeneralizedLinearRegressionModel)
S3method(print, summary.KSTest)
S3method(print, summary.RandomForestRegressionModel)
S3method(print, summary.RandomForestClassificationModel)
S3method(print, summary.GBTRegressionModel)
S3method(print, summary.GBTClassificationModel)
S3method(structField, character)
S3method(structField, jobj)
S3method(structType, jobj)
Expand Down
4 changes: 4 additions & 0 deletions R/pkg/R/generics.R
Original file line number Diff line number Diff line change
Expand Up @@ -1343,6 +1343,10 @@ setGeneric("spark.als", function(data, ...) { standardGeneric("spark.als") })
setGeneric("spark.gaussianMixture",
function(data, formula, ...) { standardGeneric("spark.gaussianMixture") })

#' @rdname spark.gbt
#' @export
setGeneric("spark.gbt", function(data, formula, ...) { standardGeneric("spark.gbt") })

#' @rdname spark.glm
#' @export
setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.glm") })
Expand Down
331 changes: 286 additions & 45 deletions R/pkg/R/mllib.R

Large diffs are not rendered by default.

68 changes: 68 additions & 0 deletions R/pkg/inst/tests/testthat/test_mllib.R
Original file line number Diff line number Diff line change
Expand Up @@ -939,4 +939,72 @@ test_that("spark.randomForest Classification", {
unlink(modelPath)
})

test_that("spark.gbt", {
# regression
data <- suppressWarnings(createDataFrame(longley))
model <- spark.gbt(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, seed = 123)
predictions <- collect(predict(model, data))
expect_equal(predictions$prediction, c(60.323, 61.122, 60.171, 61.187,
63.221, 63.639, 64.989, 63.761,
66.019, 67.857, 68.169, 66.513,
68.655, 69.564, 69.331, 70.551),
tolerance = 1e-4)
stats <- summary(model)
expect_equal(stats$numTrees, 20)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why only check numTrees? I think we should also check numFeatures, featureImportances and treeWeights at least. Any thoughts?

Copy link
Member Author

@felixcheung felixcheung Nov 7, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added. featureImportances is a bit tricky - in JVM it's a Vector and doesn't translate to something accessible in R (see SPARK-18226)

so for now featureImpoartances is converted to a string, and let's skip testing that for now

Copy link
Contributor

@yanboliang yanboliang Nov 8, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. Since there is no object represents ML Vector in SparkR currently, I'd like to convert the type of featureImportances from Vector to Array at GBTClassifierWrapper.scala.

lazy val featureImportances: Array = gbtcModel.featureImportances.toArray

Then it can be translated to R list. Users may sort or select the feature importances, so return as R list should make more sense. Any thoughts?

Copy link
Member Author

@felixcheung felixcheung Nov 8, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I tried that and it was really a SparseVector so converting to an Array made it fairly unreadable and unusable.

I think SparseVector should really map to a Map or a Properties.

expect_equal(stats$formula, "Employed ~ .")
expect_equal(stats$numFeatures, 6)
expect_equal(length(stats$treeWeights), 20)

modelPath <- tempfile(pattern = "spark-gbtRegression", fileext = ".tmp")
write.ml(model, modelPath)
expect_error(write.ml(model, modelPath))
write.ml(model, modelPath, overwrite = TRUE)
model2 <- read.ml(modelPath)
stats2 <- summary(model2)
expect_equal(stats$formula, stats2$formula)
expect_equal(stats$numFeatures, stats2$numFeatures)
expect_equal(stats$features, stats2$features)
expect_equal(stats$featureImportances, stats2$featureImportances)
expect_equal(stats$numTrees, stats2$numTrees)
expect_equal(stats$treeWeights, stats2$treeWeights)

unlink(modelPath)

# classification
# label must be binary - GBTClassifier currently only supports binary classification.
iris2 <- iris[iris$Species != "virginica", ]
data <- suppressWarnings(createDataFrame(iris2))
model <- spark.gbt(data, Species ~ Petal_Length + Petal_Width, "classification")
stats <- summary(model)
expect_equal(stats$numFeatures, 2)
expect_equal(stats$numTrees, 20)
expect_error(capture.output(stats), NA)
expect_true(length(capture.output(stats)) > 6)
predictions <- collect(predict(model, data))$prediction
# test string prediction values
expect_equal(length(grep("setosa", predictions)), 50)
expect_equal(length(grep("versicolor", predictions)), 50)

modelPath <- tempfile(pattern = "spark-gbtClassification", fileext = ".tmp")
write.ml(model, modelPath)
expect_error(write.ml(model, modelPath))
write.ml(model, modelPath, overwrite = TRUE)
model2 <- read.ml(modelPath)
stats2 <- summary(model2)
expect_equal(stats$depth, stats2$depth)
expect_equal(stats$numNodes, stats2$numNodes)
expect_equal(stats$numClasses, stats2$numClasses)

unlink(modelPath)

iris2$NumericSpecies <- ifelse(iris2$Species == "setosa", 0, 1)
df <- suppressWarnings(createDataFrame(iris2))
m <- spark.gbt(df, NumericSpecies ~ ., type = "classification")
s <- summary(m)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like we never use this line?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added a test, but this is mostly to make sure the call is not failing

# test numeric prediction values
expect_equal(iris2$NumericSpecies, as.double(collect(predict(m, df))$prediction))
expect_equal(s$numFeatures, 5)
expect_equal(s$numTrees, 20)
})

sparkR.session.stop()
Original file line number Diff line number Diff line change
@@ -0,0 +1,164 @@
/*
* 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.ml.r

import org.apache.hadoop.fs.Path
import org.json4s._
import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods._

import org.apache.spark.ml.{Pipeline, PipelineModel}
import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NominalAttribute}
import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier}
import org.apache.spark.ml.feature.{IndexToString, RFormula}
import org.apache.spark.ml.linalg.Vector
import org.apache.spark.ml.util._
import org.apache.spark.sql.{DataFrame, Dataset}

private[r] class GBTClassifierWrapper private (
val pipeline: PipelineModel,
val formula: String,
val features: Array[String]) extends MLWritable {

import GBTClassifierWrapper._

private val gbtcModel: GBTClassificationModel =
pipeline.stages(1).asInstanceOf[GBTClassificationModel]

lazy val numFeatures: Int = gbtcModel.numFeatures
lazy val featureImportances: Vector = gbtcModel.featureImportances
lazy val numTrees: Int = gbtcModel.getNumTrees
lazy val treeWeights: Array[Double] = gbtcModel.treeWeights

def summary: String = gbtcModel.toDebugString

def transform(dataset: Dataset[_]): DataFrame = {
pipeline.transform(dataset)
.drop(PREDICTED_LABEL_INDEX_COL)
.drop(gbtcModel.getFeaturesCol)
}

override def write: MLWriter = new
GBTClassifierWrapper.GBTClassifierWrapperWriter(this)
}

private[r] object GBTClassifierWrapper extends MLReadable[GBTClassifierWrapper] {

val PREDICTED_LABEL_INDEX_COL = "pred_label_idx"
val PREDICTED_LABEL_COL = "prediction"

def fit( // scalastyle:ignore
data: DataFrame,
formula: String,
maxDepth: Int,
maxBins: Int,
maxIter: Int,
stepSize: Double,
minInstancesPerNode: Int,
minInfoGain: Double,
checkpointInterval: Int,
lossType: String,
seed: String,
subsamplingRate: Double,
maxMemoryInMB: Int,
cacheNodeIds: Boolean): GBTClassifierWrapper = {

val rFormula = new RFormula()
.setFormula(formula)
.setForceIndexLabel(true)
RWrapperUtils.checkDataColumns(rFormula, data)
val rFormulaModel = rFormula.fit(data)

// get feature names from output schema
val schema = rFormulaModel.transform(data).schema
val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol))
.attributes.get
val features = featureAttrs.map(_.name.get)

// get label names from output schema
val labelAttr = Attribute.fromStructField(schema(rFormulaModel.getLabelCol))
.asInstanceOf[NominalAttribute]
val labels = labelAttr.values.get

// assemble and fit the pipeline
val rfc = new GBTClassifier()
.setMaxDepth(maxDepth)
.setMaxBins(maxBins)
.setMaxIter(maxIter)
.setStepSize(stepSize)
.setMinInstancesPerNode(minInstancesPerNode)
.setMinInfoGain(minInfoGain)
.setCheckpointInterval(checkpointInterval)
.setLossType(lossType)
.setSubsamplingRate(subsamplingRate)
.setMaxMemoryInMB(maxMemoryInMB)
.setCacheNodeIds(cacheNodeIds)
.setFeaturesCol(rFormula.getFeaturesCol)
.setPredictionCol(PREDICTED_LABEL_INDEX_COL)
if (seed != null && seed.length > 0) rfc.setSeed(seed.toLong)

val idxToStr = new IndexToString()
.setInputCol(PREDICTED_LABEL_INDEX_COL)
.setOutputCol(PREDICTED_LABEL_COL)
.setLabels(labels)

val pipeline = new Pipeline()
.setStages(Array(rFormulaModel, rfc, idxToStr))
.fit(data)

new GBTClassifierWrapper(pipeline, formula, features)
}

override def read: MLReader[GBTClassifierWrapper] = new GBTClassifierWrapperReader

override def load(path: String): GBTClassifierWrapper = super.load(path)

class GBTClassifierWrapperWriter(instance: GBTClassifierWrapper)
extends MLWriter {

override protected def saveImpl(path: String): Unit = {
val rMetadataPath = new Path(path, "rMetadata").toString
val pipelinePath = new Path(path, "pipeline").toString

val rMetadata = ("class" -> instance.getClass.getName) ~
("formula" -> instance.formula) ~
("features" -> instance.features.toSeq)
val rMetadataJson: String = compact(render(rMetadata))

sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath)
instance.pipeline.save(pipelinePath)
}
}

class GBTClassifierWrapperReader extends MLReader[GBTClassifierWrapper] {

override def load(path: String): GBTClassifierWrapper = {
implicit val format = DefaultFormats
val rMetadataPath = new Path(path, "rMetadata").toString
val pipelinePath = new Path(path, "pipeline").toString
val pipeline = PipelineModel.load(pipelinePath)

val rMetadataStr = sc.textFile(rMetadataPath, 1).first()
val rMetadata = parse(rMetadataStr)
val formula = (rMetadata \ "formula").extract[String]
val features = (rMetadata \ "features").extract[Array[String]]

new GBTClassifierWrapper(pipeline, formula, features)
}
}
}
Loading