Skip to content

Commit

Permalink
[SPARK-12160][MLLIB] Use SQLContext.getOrCreate in MLlib
Browse files Browse the repository at this point in the history
Switched from using SQLContext constructor to using getOrCreate, mainly in model save/load methods.

This covers all instances in spark.mllib.  There were no uses of the constructor in spark.ml.

CC: mengxr yhuai

Author: Joseph K. Bradley <[email protected]>

Closes apache#10161 from jkbradley/mllib-sqlcontext-fix.

Conflicts:
	mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala
  • Loading branch information
jkbradley committed Dec 8, 2015
1 parent 3868ab6 commit acf1e55
Show file tree
Hide file tree
Showing 12 changed files with 27 additions and 27 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1149,7 +1149,7 @@ private[python] class PythonMLLibAPI extends Serializable {
def getIndexedRows(indexedRowMatrix: IndexedRowMatrix): DataFrame = {
// We use DataFrames for serialization of IndexedRows to Python,
// so return a DataFrame.
val sqlContext = new SQLContext(indexedRowMatrix.rows.sparkContext)
val sqlContext = SQLContext.getOrCreate(indexedRowMatrix.rows.sparkContext)
sqlContext.createDataFrame(indexedRowMatrix.rows)
}

Expand All @@ -1159,7 +1159,7 @@ private[python] class PythonMLLibAPI extends Serializable {
def getMatrixEntries(coordinateMatrix: CoordinateMatrix): DataFrame = {
// We use DataFrames for serialization of MatrixEntry entries to
// Python, so return a DataFrame.
val sqlContext = new SQLContext(coordinateMatrix.entries.sparkContext)
val sqlContext = SQLContext.getOrCreate(coordinateMatrix.entries.sparkContext)
sqlContext.createDataFrame(coordinateMatrix.entries)
}

Expand All @@ -1169,7 +1169,7 @@ private[python] class PythonMLLibAPI extends Serializable {
def getMatrixBlocks(blockMatrix: BlockMatrix): DataFrame = {
// We use DataFrames for serialization of sub-matrix blocks to
// Python, so return a DataFrame.
val sqlContext = new SQLContext(blockMatrix.blocks.sparkContext)
val sqlContext = SQLContext.getOrCreate(blockMatrix.blocks.sparkContext)
sqlContext.createDataFrame(blockMatrix.blocks)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] {
modelType: String)

def save(sc: SparkContext, path: String, data: Data): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// Create JSON metadata.
Expand All @@ -208,7 +208,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] {

@Since("1.3.0")
def load(sc: SparkContext, path: String): NaiveBayesModel = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
// Load Parquet data.
val dataRDD = sqlContext.read.parquet(dataPath(path))
// Check schema explicitly since erasure makes it hard to use match-case for checking.
Expand Down Expand Up @@ -239,7 +239,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] {
theta: Array[Array[Double]])

def save(sc: SparkContext, path: String, data: Data): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// Create JSON metadata.
Expand All @@ -254,7 +254,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] {
}

def load(sc: SparkContext, path: String): NaiveBayesModel = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
// Load Parquet data.
val dataRDD = sqlContext.read.parquet(dataPath(path))
// Check schema explicitly since erasure makes it hard to use match-case for checking.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ private[classification] object GLMClassificationModel {
weights: Vector,
intercept: Double,
threshold: Option[Double]): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// Create JSON metadata.
Expand All @@ -74,7 +74,7 @@ private[classification] object GLMClassificationModel {
*/
def loadData(sc: SparkContext, path: String, modelClass: String): Data = {
val datapath = Loader.dataPath(path)
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(datapath)
val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1)
assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] {
weights: Array[Double],
gaussians: Array[MultivariateGaussian]): Unit = {

val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// Create JSON metadata.
Expand All @@ -166,7 +166,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] {

def load(sc: SparkContext, path: String): GaussianMixtureModel = {
val dataPath = Loader.dataPath(path)
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val dataFrame = sqlContext.read.parquet(dataPath)
val dataArray = dataFrame.select("weight", "mu", "sigma").collect()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ object KMeansModel extends Loader[KMeansModel] {
val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel"

def save(sc: SparkContext, model: KMeansModel, path: String): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._
val metadata = compact(render(
("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k)))
Expand All @@ -137,7 +137,7 @@ object KMeansModel extends Loader[KMeansModel] {

def load(sc: SparkContext, path: String): KMeansModel = {
implicit val formats = DefaultFormats
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path)
assert(className == thisClassName)
assert(formatVersion == thisFormatVersion)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode

@Since("1.4.0")
def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

val metadata = compact(render(
Expand All @@ -87,7 +87,7 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode
@Since("1.4.0")
def load(sc: SparkContext, path: String): PowerIterationClusteringModel = {
implicit val formats = DefaultFormats
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)

val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path)
assert(className == thisClassName)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -588,7 +588,7 @@ object Word2VecModel extends Loader[Word2VecModel] {

def load(sc: SparkContext, path: String): Word2VecModel = {
val dataPath = Loader.dataPath(path)
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val dataFrame = sqlContext.read.parquet(dataPath)

val dataArray = dataFrame.select("word", "vector").collect()
Expand All @@ -602,7 +602,7 @@ object Word2VecModel extends Loader[Word2VecModel] {

def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]): Unit = {

val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

val vectorSize = model.values.head.size
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,7 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] {
*/
def save(model: MatrixFactorizationModel, path: String): Unit = {
val sc = model.userFeatures.sparkContext
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._
val metadata = compact(render(
("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rank" -> model.rank)))
Expand All @@ -364,7 +364,7 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] {

def load(sc: SparkContext, path: String): MatrixFactorizationModel = {
implicit val formats = DefaultFormats
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val (className, formatVersion, metadata) = loadMetadata(sc, path)
assert(className == thisClassName)
assert(formatVersion == thisFormatVersion)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] {
boundaries: Array[Double],
predictions: Array[Double],
isotonic: Boolean): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)

val metadata = compact(render(
("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~
Expand All @@ -201,7 +201,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] {
}

def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(dataPath(path))

checkSchema[Data](dataRDD.schema)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ private[regression] object GLMRegressionModel {
modelClass: String,
weights: Vector,
intercept: Double): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// Create JSON metadata.
Expand All @@ -71,7 +71,7 @@ private[regression] object GLMRegressionModel {
*/
def loadData(sc: SparkContext, path: String, modelClass: String, numFeatures: Int): Data = {
val datapath = Loader.dataPath(path)
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(datapath)
val dataArray = dataRDD.select("weights", "intercept").take(1)
assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging {
}

def save(sc: SparkContext, path: String, model: DecisionTreeModel): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// SPARK-6120: We do a hacky check here so users understand why save() is failing
Expand Down Expand Up @@ -244,7 +244,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging {

def load(sc: SparkContext, path: String, algo: String, numNodes: Int): DecisionTreeModel = {
val datapath = Loader.dataPath(path)
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
// Load Parquet data.
val dataRDD = sqlContext.read.parquet(datapath)
// Check schema explicitly since erasure makes it hard to use match-case for checking.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -413,7 +413,7 @@ private[tree] object TreeEnsembleModel extends Logging {
case class EnsembleNodeData(treeId: Int, node: NodeData)

def save(sc: SparkContext, path: String, model: TreeEnsembleModel, className: String): Unit = {
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._

// SPARK-6120: We do a hacky check here so users understand why save() is failing
Expand Down Expand Up @@ -473,7 +473,7 @@ private[tree] object TreeEnsembleModel extends Logging {
path: String,
treeAlgo: String): Array[DecisionTreeModel] = {
val datapath = Loader.dataPath(path)
val sqlContext = new SQLContext(sc)
val sqlContext = SQLContext.getOrCreate(sc)
val nodes = sqlContext.read.parquet(datapath).map(NodeData.apply)
val trees = constructTrees(nodes)
trees.map(new DecisionTreeModel(_, Algo.fromString(treeAlgo)))
Expand Down

0 comments on commit acf1e55

Please sign in to comment.