diff --git a/data/mllib/sample_kmeans_data.txt b/data/mllib/sample_kmeans_data.txt new file mode 100644 index 0000000000000000000000000000000000000000..50013776b182ae7dbba4a877c6cb3e176bae88bd --- /dev/null +++ b/data/mllib/sample_kmeans_data.txt @@ -0,0 +1,6 @@ +0 1:0.0 2:0.0 3:0.0 +1 1:0.1 2:0.1 3:0.1 +2 1:0.2 2:0.2 3:0.2 +3 1:9.0 2:9.0 3:9.0 +4 1:9.1 2:9.1 3:9.1 +5 1:9.2 2:9.2 3:9.2 diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 440c455cd077c39d06a399856f6fa10f8f54e3f4..1245b8bbc844ccc674cccf746e7e7ffaff35b6ca 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -104,4 +104,39 @@ Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/LDA.html) f {% include_example java/org/apache/spark/examples/ml/JavaLDAExample.java %} </div> -</div> \ No newline at end of file +</div> + +## Bisecting k-means + + +Bisecting k-means is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering) using a +divisive (or "top-down") approach: all observations start in one cluster, and splits are performed recursively as one +moves down the hierarchy. + +Bisecting K-means can often be much faster than regular K-means, but it will generally produce a different clustering. + +`BisectingKMeans` is implemented as an `Estimator` and generates a `BisectingKMeansModel` as the base model. + +### Example + +<div class="codetabs"> + +<div data-lang="scala" markdown="1"> +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.BisectingKMeans) for more details. + +{% include_example scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala %} +</div> + +<div data-lang="java" markdown="1"> +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/BisectingKMeans.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java %} +</div> + +<div data-lang="python" markdown="1"> +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.BisectingKMeans) for more details. + +{% include_example python/ml/bisecting_k_means_example.py %} +</div> + +</div> diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 810ad905c56af90e4c53cc08cd13e1f3bb6bb254..62871448e36f54631c7d616ffd34d020c7383a4d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -17,27 +17,22 @@ package org.apache.spark.examples.ml; -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; // $example off$ +import org.apache.spark.sql.SparkSession; /** - * An example demonstrating a bisecting k-means clustering. + * An example demonstrating bisecting k-means clustering. + * Run with + * <pre> + * bin/run-example ml.JavaBisectingKMeansExample + * </pre> */ public class JavaBisectingKMeansExample { @@ -48,30 +43,22 @@ public class JavaBisectingKMeansExample { .getOrCreate(); // $example on$ - List<Row> data = Arrays.asList( - RowFactory.create(Vectors.dense(0.1, 0.1, 0.1)), - RowFactory.create(Vectors.dense(0.3, 0.3, 0.25)), - RowFactory.create(Vectors.dense(0.1, 0.1, -0.1)), - RowFactory.create(Vectors.dense(20.3, 20.1, 19.9)), - RowFactory.create(Vectors.dense(20.2, 20.1, 19.7)), - RowFactory.create(Vectors.dense(18.9, 20.0, 19.7)) - ); - - StructType schema = new StructType(new StructField[]{ - new StructField("features", new VectorUDT(), false, Metadata.empty()), - }); - - Dataset<Row> dataset = spark.createDataFrame(data, schema); + // Loads data. + Dataset<Row> dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); - BisectingKMeans bkm = new BisectingKMeans().setK(2); + // Trains a bisecting k-means model. + BisectingKMeans bkm = new BisectingKMeans().setK(2).setSeed(1); BisectingKMeansModel model = bkm.fit(dataset); - System.out.println("Compute Cost: " + model.computeCost(dataset)); + // Evaluate clustering. + double cost = model.computeCost(dataset); + System.out.println("Within Set Sum of Squared Errors = " + cost); - Vector[] clusterCenters = model.clusterCenters(); - for (int i = 0; i < clusterCenters.length; i++) { - Vector clusterCenter = clusterCenters[i]; - System.out.println("Cluster Center " + i + ": " + clusterCenter); + // Shows the result. + System.out.println("Cluster Centers: "); + Vector[] centers = model.clusterCenters(); + for (Vector center : centers) { + System.out.println(center); } // $example off$ diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 540a4bc3e4bf98641823fe4a44b22f6cd317872a..ee0399ac5eb20cf5f87a0a726e6690eb287570fd 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -18,15 +18,14 @@ from __future__ import print_function # $example on$ -from pyspark.ml.clustering import BisectingKMeans, BisectingKMeansModel -from pyspark.mllib.linalg import VectorUDT, _convert_to_vector, Vectors -from pyspark.mllib.linalg import Vectors -from pyspark.sql.types import Row +from pyspark.ml.clustering import BisectingKMeans # $example off$ from pyspark.sql import SparkSession """ -A simple example demonstrating a bisecting k-means clustering. +An example demonstrating bisecting k-means clustering. +Run with: + bin/spark-submit examples/src/main/python/ml/bisecting_k_means_example.py """ if __name__ == "__main__": @@ -36,21 +35,20 @@ if __name__ == "__main__": .getOrCreate() # $example on$ - data = spark.read.text("data/mllib/kmeans_data.txt").rdd - parsed = data\ - .map(lambda row: Row(features=Vectors.dense([float(x) for x in row.value.split(' ')]))) - training = spark.createDataFrame(parsed) + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") - kmeans = BisectingKMeans().setK(2).setSeed(1).setFeaturesCol("features") + # Trains a bisecting k-means model. + bkm = BisectingKMeans().setK(2).setSeed(1) + model = bkm.fit(dataset) - model = kmeans.fit(training) + # Evaluate clustering. + cost = model.computeCost(dataset) + print("Within Set Sum of Squared Errors = " + str(cost)) - # Evaluate clustering - cost = model.computeCost(training) - print("Bisecting K-means Cost = " + str(cost)) - - centers = model.clusterCenters() + # Shows the result. print("Cluster Centers: ") + centers = model.clusterCenters() for center in centers: print(center) # $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala new file mode 100644 index 0000000000000000000000000000000000000000..5f8f2c99cbaf4e3206bd263520de7070f056e3b1 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala @@ -0,0 +1,65 @@ +/* + * 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.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.BisectingKMeans +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating bisecting k-means clustering. + * Run with + * {{{ + * bin/run-example ml.BisectingKMeansExample + * }}} + */ +object BisectingKMeansExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession + .builder + .appName("BisectingKMeansExample") + .getOrCreate() + + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains a bisecting k-means model. + val bkm = new BisectingKMeans().setK(2).setSeed(1) + val model = bkm.fit(dataset) + + // Evaluate clustering. + val cost = model.computeCost(dataset) + println(s"Within Set Sum of Squared Errors = $cost") + + // Shows the result. + println("Cluster Centers: ") + val centers = model.clusterCenters + centers.foreach(println) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println +