Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-10017] [MLlib]: ML model broadcasts should be stored in private vars: mllib NaiveBayes #8241

Closed
wants to merge 7 commits into from
Closed
Show file tree
Hide file tree
Changes from 6 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
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,15 @@ import java.lang.{Iterable => JIterable}

import scala.collection.JavaConverters._

import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods._

import org.apache.spark.{Logging, SparkContext, SparkException}
import org.apache.spark.annotation.Since
import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, SparseVector, Vector}
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.util.{Loader, Saveable}
import org.apache.spark.mllib.util.{Broadcastable, Loader, Saveable}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.json4s.JsonDSL._
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do we need these imports now?

Copy link
Author

Choose a reason for hiding this comment

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

@feynmanliang These json* imports were always there. They were simply reordered by the import order plugin I used for IntelliJ. The only additional import is for the Broadcastable trait.

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh, those need to go back to where they were before, see style guide

Copy link
Author

Choose a reason for hiding this comment

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

Cool, ill take care of that.

import org.json4s.jackson.JsonMethods._

/**
* Model for Naive Bayes Classifiers.
Expand All @@ -47,7 +46,7 @@ class NaiveBayesModel private[spark] (
@Since("0.9.0") val pi: Array[Double],
@Since("0.9.0") val theta: Array[Array[Double]],
@Since("1.4.0") val modelType: String)
extends ClassificationModel with Serializable with Saveable {
extends ClassificationModel with Serializable with Saveable with Broadcastable[NaiveBayesModel] {

import NaiveBayes.{Bernoulli, Multinomial, supportedModelTypes}

Expand Down Expand Up @@ -86,9 +85,10 @@ class NaiveBayesModel private[spark] (

@Since("1.0.0")
override def predict(testData: RDD[Vector]): RDD[Double] = {
val bcModel = testData.context.broadcast(this)
val sc = testData.sparkContext
val lclBcModel = getBroadcastModel(sc, this)
testData.mapPartitions { iter =>
val model = bcModel.value
val model = lclBcModel.value
iter.map(model.predict)
}
}
Expand All @@ -112,9 +112,10 @@ class NaiveBayesModel private[spark] (
*/
@Since("1.5.0")
def predictProbabilities(testData: RDD[Vector]): RDD[Vector] = {
val bcModel = testData.context.broadcast(this)
val sc = testData.sparkContext
val lclBcModel = getBroadcastModel(sc, this)
testData.mapPartitions { iter =>
val model = bcModel.value
val model = lclBcModel.value
iter.map(model.predictProbabilities)
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.mllib.util

import scala.reflect.ClassTag

import org.apache.spark.SparkContext
import org.apache.spark.broadcast.Broadcast

private[spark] trait Broadcastable[T] {

private var bcModel: Option[Broadcast[T]] = None

/**
* Checks whether the model object is already broadcast and returns the reference.
* If not, then broadcasts the model and returns a reference
* @param sc SparkContext that will be used for the broadcast
* @param modelToBc Model object to broadcast
* @return the broadcast model
*/
def getBroadcastModel(sc: SparkContext, modelToBc: T)
(implicit ev: ClassTag[T]) : Broadcast[T] = {
Copy link
Member

Choose a reason for hiding this comment

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

Was the ClassTag necessary to get this to work? If you want to make sure T is a ClassTag, can you do

def getBroadcastModel[T: ClassTag](sc ...): Broadcast[T]

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it might be better to keep parameterization on the entire trait rather than just getBroadcastModel i.e. remove the implicit here and add a context bound trait Broadcastable[T : ClassTag] { up in the trait declaration

Copy link
Member

Choose a reason for hiding this comment

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

+1 that's better

bcModel match {
case None => bcModel = Some(sc.broadcast(modelToBc))
case _ =>
}
bcModel.get
}

}