|
17 | 17 |
|
18 | 18 | package org.apache.spark.mllib.clustering
|
19 | 19 |
|
| 20 | +import org.json4s._ |
| 21 | +import org.json4s.JsonDSL._ |
| 22 | +import org.json4s.jackson.JsonMethods._ |
| 23 | + |
20 | 24 | import org.apache.spark.api.java.JavaRDD
|
21 |
| -import org.apache.spark.rdd.RDD |
22 |
| -import org.apache.spark.SparkContext._ |
23 | 25 | import org.apache.spark.mllib.linalg.Vector
|
| 26 | +import org.apache.spark.mllib.util.{Loader, Saveable} |
| 27 | +import org.apache.spark.rdd.RDD |
| 28 | +import org.apache.spark.SparkContext |
| 29 | +import org.apache.spark.sql.SQLContext |
| 30 | +import org.apache.spark.sql.Row |
24 | 31 |
|
25 | 32 | /**
|
26 | 33 | * A clustering model for K-means. Each point belongs to the cluster with the closest center.
|
27 | 34 | */
|
28 |
| -class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { |
| 35 | +class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable { |
29 | 36 |
|
30 | 37 | /** Total number of clusters. */
|
31 | 38 | def k: Int = clusterCenters.length
|
@@ -58,4 +65,59 @@ class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable {
|
58 | 65 |
|
59 | 66 | private def clusterCentersWithNorm: Iterable[VectorWithNorm] =
|
60 | 67 | clusterCenters.map(new VectorWithNorm(_))
|
| 68 | + |
| 69 | + override def save(sc: SparkContext, path: String): Unit = { |
| 70 | + KMeansModel.SaveLoadV1_0.save(sc, this, path) |
| 71 | + } |
| 72 | + |
| 73 | + override protected def formatVersion: String = "1.0" |
| 74 | +} |
| 75 | + |
| 76 | +object KMeansModel extends Loader[KMeansModel] { |
| 77 | + override def load(sc: SparkContext, path: String): KMeansModel = { |
| 78 | + KMeansModel.SaveLoadV1_0.load(sc, path) |
| 79 | + } |
| 80 | + |
| 81 | + private case class Cluster(id: Int, point: Vector) |
| 82 | + |
| 83 | + private object Cluster { |
| 84 | + def apply(r: Row): Cluster = { |
| 85 | + Cluster(r.getInt(0), r.getAs[Vector](1)) |
| 86 | + } |
| 87 | + } |
| 88 | + |
| 89 | + private[clustering] |
| 90 | + object SaveLoadV1_0 { |
| 91 | + |
| 92 | + private val thisFormatVersion = "1.0" |
| 93 | + |
| 94 | + private[clustering] |
| 95 | + val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel" |
| 96 | + |
| 97 | + def save(sc: SparkContext, model: KMeansModel, path: String): Unit = { |
| 98 | + val sqlContext = new SQLContext(sc) |
| 99 | + import sqlContext.implicits._ |
| 100 | + val metadata = compact(render( |
| 101 | + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) |
| 102 | + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) |
| 103 | + val dataRDD = sc.parallelize(model.clusterCenters.zipWithIndex).map { case (point, id) => |
| 104 | + Cluster(id, point) |
| 105 | + }.toDF() |
| 106 | + dataRDD.saveAsParquetFile(Loader.dataPath(path)) |
| 107 | + } |
| 108 | + |
| 109 | + def load(sc: SparkContext, path: String): KMeansModel = { |
| 110 | + implicit val formats = DefaultFormats |
| 111 | + val sqlContext = new SQLContext(sc) |
| 112 | + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) |
| 113 | + assert(className == thisClassName) |
| 114 | + assert(formatVersion == thisFormatVersion) |
| 115 | + val k = (metadata \ "k").extract[Int] |
| 116 | + val centriods = sqlContext.parquetFile(Loader.dataPath(path)) |
| 117 | + Loader.checkSchema[Cluster](centriods.schema) |
| 118 | + val localCentriods = centriods.map(Cluster.apply).collect() |
| 119 | + assert(k == localCentriods.size) |
| 120 | + new KMeansModel(localCentriods.sortBy(_.id).map(_.point)) |
| 121 | + } |
| 122 | + } |
61 | 123 | }
|
0 commit comments