[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-12 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/18538


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138255937
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,438 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"silhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("silhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (silhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "silhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
--- End diff --

We should support all numeric type for prediction column, not only integer. 
```
SchemaUtils.checkNumericType(schema, $(labelCol))
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138256035
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,438 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"silhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("silhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (silhouette)",
+  allowedParams
+)
--- End diff --

Reorg as:
```
val metricName: Param[String] = {
val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
new Param(
  this, "metricName", "metric name in evaluation (squaredSilhouette)", 
allowedParams)
}
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138255648
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,438 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
--- End diff --

Usually we leave a blank line under ```:: Experimental ::```.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138255474
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,438 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"silhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("silhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (silhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "silhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+$(metricName) match {
+  case "silhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
--- End diff --

Reorg as:
```
$(metricName) match {
  case "squaredSilhouette" =>
SquaredEuclideanSilhouette.computeSilhouetteScore(
  dataset, $(predictionCol), $(featuresCol))
}
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138090203
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
+  "Number of clusters must be greater than one.")
+
+$(metricName) match {
+  case "squaredSilhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
+}
+  }
+}
+
+
+@Since("2.3.0")
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  @Since("2.3.0")
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138025640
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
--- End diff --

Move this check to L418, in case another unnecessary computation for most 
of the cases(cluster size > 1). See my comment at L418.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138024385
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
+  "Number of clusters must be greater than one.")
+
+$(metricName) match {
+  case "squaredSilhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
+}
+  }
+}
+
+
+@Since("2.3.0")
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  @Since("2.3.0")
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138025184
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
+  "Number of clusters must be greater than one.")
+
+$(metricName) match {
+  case "squaredSilhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
+}
+  }
+}
+
+
+@Since("2.3.0")
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  @Since("2.3.0")
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138027427
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
--- End diff --

I'd suggest the metric name is ```silhouette```, since we may add 
silhouette for other distance, then we can add another param like 
```distance``` to control that. The param ```metricName``` should not bind to 
any distance computation way. There are lots of other metrics for clustering 
algorithms, like 
[these](http://scikit-learn.org/stable/modules/classes.html#clustering-metrics) 
in sklearn. We would not add all of them for MLlib, but we may add part of them 
in the future.
cc @jkbradley @MLnick @WeichenXu123 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138021102
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
+  "Number of clusters must be greater than one.")
+
+$(metricName) match {
+  case "squaredSilhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
+}
+  }
+}
+
+
+@Since("2.3.0")
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  @Since("2.3.0")
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138024573
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
+  "Number of clusters must be greater than one.")
+
+$(metricName) match {
+  case "squaredSilhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
+}
+  }
+}
+
+
+@Since("2.3.0")
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  @Since("2.3.0")
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-11 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r138023290
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,437 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  @Since("2.3.0")
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  @Since("2.3.0")
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  @Since("2.3.0")
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  @Since("2.3.0")
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  @Since("2.3.0")
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+// Silhouette is reasonable only when the number of clusters is grater 
then 1
+assert(dataset.select($(predictionCol)).distinct().count() > 1,
+  "Number of clusters must be greater than one.")
+
+$(metricName) match {
+  case "squaredSilhouette" => 
SquaredEuclideanSilhouette.computeSilhouetteScore(
+dataset,
+$(predictionCol),
+$(featuresCol)
+  )
+}
+  }
+}
+
+
+@Since("2.3.0")
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  @Since("2.3.0")
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137310194
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+
+assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-10)
+  }
+
--- End diff --

yes, I agree. Thanks.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137278367
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+
+assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-10)
+  }
+
--- End diff --

Yeah, I support to keep consistent result. Otherwise, any real value is a 
confused result. What do you think of it? Thanks.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137275123
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+
+assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-10)
+  }
+
--- End diff --

Actually sklearn throws an exception in this case. Should we do the same? 
Thanks.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137261873
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
--- End diff --

@zhengruifeng I think you asked me to remove it, any concern if I add it 
back? Thanks..


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137261509
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
--- End diff --

No problem at all, it is just to know which is the way to go. Then I'll add 
it back, thanks.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137253446
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
--- End diff --

Sorry if the comment was left by me. Anyway, I think we should add it, 
since this class is ```ClusteringEvaluator``` rather than silhouette metric, 
users should know which metric they are using. And we will support more metrics 
in the future. Thanks.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137251472
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137251320
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137250195
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137249408
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
--- End diff --

At the beginning it was like that but then in some comments I had been 
asked to remove it as it is useless at the moment.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239650
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
--- End diff --

```@Since("2.3.0") ```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137240370
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+
+assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-10)
--- End diff --

Check with tolerance 1e-5 is good enough.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137224923
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239744
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
--- End diff --

```@Since("2.3.0")```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137242981
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+
+assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-10)
+  }
+
--- End diff --

It's better to add another corner case: single cluster. We should guarantee 
it output consistent result with sklearn. You can just select one cluster from 
the iris dataset and test it. 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137180832
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239772
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
--- End diff --

```@Since("2.3.0")```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137178736
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137238642
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137226104
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137180329
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137226738
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239566
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
--- End diff --

Could we add a param ```metricName``` like other evaluator? It can only 
support ```silhouette``` currently, but we may add other metric in the future. 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239933
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
--- End diff --

```@Since("2.3.0")``` 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137180194
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137178071
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137226969
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239478
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
--- End diff --

```class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val 
uid: String)```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137226318
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137242127
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.ml.util.TestingUtils._
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+
+assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-10)
+  }
+
+}
+
+object ClusteringEvaluatorSuite {
+  def irisDataset(spark: SparkSession): DataFrame = {
+import spark.implicits._
+
+val irisCsvPath = Thread.currentThread()
+  .getContextClassLoader
+  .getResource("test-data/iris.csv")
+  .toString
+
+spark.sparkContext
+  .textFile(irisCsvPath)
+  .map {
+line =>
+  val splits = line.split(",")
+  ClusteringEvaluationTestData(
+Vectors.dense(splits.take(splits.length-1).map(_.toDouble)),
+splits(splits.length-1).toInt
+  )
+  }
+  .toDF()
--- End diff --

Can we store the test data as libsvm format rather than csv? Then we can 
use ```spark.read.format("libsvm").load(irisPath)``` to load it to a DataFrame 
with two columns: features and label.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137239906
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
--- End diff --

```@Since("2.3.0")``` 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137175816
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
--- End diff --

```of to``` -> ```of `i` to```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-06 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r137178833
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,396 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * :: Experimental ::
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   $$
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ *   $$
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   $$
+ *   s_{i}= \begin{cases}
+ *   1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\
+ *   \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases}
+ *   $$
+ * 
+ *
+ * where `$a_{i}$` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `$b_{i}$` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `$a_{i}$` can be interpreted as as how well `i` is assigned to its 
cluster
+ * (the smaller the value, the better the assignment), while `$b_{i}$` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 
compute
+ * the distance of each couple of points in the dataset. Since 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-01 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136573932
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,395 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
--- End diff --

it is added in the line below, despite the comment is not considered 
outdated by github. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-01 Thread WeichenXu123
Github user WeichenXu123 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136536168
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,91 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val result = BigDecimal(0.6564679231)
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+val actual = BigDecimal(evaluator.evaluate(iris))
+  .setScale(10, BigDecimal.RoundingMode.HALF_UP)
+
+assertResult(result)(actual)
--- End diff --

You can use `A ~== B relTol 1e-10`. No need `BigDecimal` I think.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-09-01 Thread WeichenXu123
Github user WeichenXu123 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136532646
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,395 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
--- End diff --

add `:: Experimental ::` for doc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136373078
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   s_{i}=\left\{ \begin{tabular}{cc}
+ *   $1-\frac{a_{i}}{b_{i}}$ & if $a_{i} \leq b_{i}$ \\
+ *   $\frac{b_{i}}{a_{i}}-1$ & if $a_{i} \gt b_{i}$
--- End diff --

thank you! You're always nice. Just fixed everything, thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136333104
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
--- End diff --

BTW, we have necessary docs at ```object SquaredEuclideanSilhouette``` to 
explain our proposed algorithm, so we can remove this. Usually we only refer to 
public publication.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136332399
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   s_{i}=\left\{ \begin{tabular}{cc}
+ *   $1-\frac{a_{i}}{b_{i}}$ & if $a_{i} \leq b_{i}$ \\
+ *   $\frac{b_{i}}{a_{i}}-1$ & if $a_{i} \gt b_{i}$
--- End diff --

1, Remove ```private[evaluation]``` from ```object 
SquaredEuclideanSilhouette```. We only generate docs for public APIs, the doc 
of private APIs are used for developers to understand code.
2, ```cd docs```
3, Run ```jekyll build```
4, Then you can get API docs under ```docs/_site/api/scala/index.html```, 
try to search ```SquaredEuclideanSilhouette```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136311477
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   s_{i}=\left\{ \begin{tabular}{cc}
+ *   $1-\frac{a_{i}}{b_{i}}$ & if $a_{i} \leq b_{i}$ \\
+ *   $\frac{b_{i}}{a_{i}}-1$ & if $a_{i} \gt b_{i}$
--- End diff --

thanks @yanboliang, may you please tell me how to check the generated doc? 
thank you!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136306135
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   s_{i}=\left\{ \begin{tabular}{cc}
+ *   $1-\frac{a_{i}}{b_{i}}$ & if $a_{i} \leq b_{i}$ \\
+ *   $\frac{b_{i}}{a_{i}}-1$ & if $a_{i} \gt b_{i}$
+ * 
+ *
+ * where `a(i)` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `b(i)` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `a(i)` can be interpreted as as how well `i` is assigned to its cluster
+ * (the smaller the value, the better the assignment), while `b(i)` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136304803
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
--- End diff --

The latex formula should be surrounded by $$, change here and other places 
as:
```

$$
s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
$$

```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136305238
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   s_{i}=\left\{ \begin{tabular}{cc}
+ *   $1-\frac{a_{i}}{b_{i}}$ & if $a_{i} \leq b_{i}$ \\
+ *   $\frac{b_{i}}{a_{i}}-1$ & if $a_{i} \gt b_{i}$
+ * 
+ *
+ * where `a(i)` is the average dissimilarity of `i` with all other data
+ * within the same cluster, `b(i)` is the lowest average dissimilarity
+ * of to any other cluster, of which `i` is not a member.
+ * `a(i)` can be interpreted as as how well `i` is assigned to its cluster
+ * (the smaller the value, the better the assignment), while `b(i)` is
+ * a measure of how well `i` has not been assigned to its "neighboring 
cluster",
+ * ie. the nearest cluster to `i`.
+ *
+ * Unfortunately, the naive implementation of the algorithm requires to 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-31 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r136305819
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,379 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.ParamMap
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * The metric computes the Silhouette measure
+ * using the squared Euclidean distance.
+ *
+ * The Silhouette is a measure for the validation
+ * of the consistency within clusters. It ranges
+ * between 1 and -1, where a value close to 1
+ * means that the points in a cluster are close
+ * to the other points in the same cluster and
+ * far from the points of the other clusters.
+ *
+ * The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ * in this document.
+ */
+@Experimental
+@Since("2.3.0")
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("cluEval"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  @Since("2.3.0")
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  @Since("2.3.0")
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+SquaredEuclideanSilhouette.computeSilhouetteScore(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+  }
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+
+/**
+ * SquaredEuclideanSilhouette computes the average of the
+ * Silhouette over all the data of the dataset, which is
+ * a measure of how appropriately the data have been clustered.
+ *
+ * The Silhouette for each point `i` is defined as:
+ *
+ * 
+ *   s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}}
+ * 
+ *
+ * which can be rewritten as
+ *
+ * 
+ *   s_{i}=\left\{ \begin{tabular}{cc}
+ *   $1-\frac{a_{i}}{b_{i}}$ & if $a_{i} \leq b_{i}$ \\
+ *   $\frac{b_{i}}{a_{i}}-1$ & if $a_{i} \gt b_{i}$
--- End diff --

There is syntax error in this latex formula, I checked the generated doc 
and found it can't show correctly. Or you can paste this formula into 
http://www.hostmath.com/ to check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-22 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r134456779
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,91 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val result = BigDecimal(0.6564679231)
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+val actual = BigDecimal(evaluator.evaluate(iris))
+  .setScale(10, BigDecimal.RoundingMode.HALF_UP)
+
+assertResult(result)(actual)
+  }
+
+}
+
+object ClusteringEvaluatorSuite {
+  def irisDataset(spark: SparkSession): DataFrame = {
+import spark.implicits._
+
+val irisCsvPath = Thread.currentThread()
+  .getContextClassLoader
+  .getResource("test-data/iris.csv")
+  .toString
--- End diff --

There was a discussion about this in the outdated comments. The main reason 
to avoid test data generation in my point of view is that the generated data 
must be clustered before running the Silhouette.
The iris dataset is a well-known one and contains already clustered data. 
Thus it seemed the best option.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-22 Thread WeichenXu123
Github user WeichenXu123 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r134449164
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,91 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+
+private[ml] case class ClusteringEvaluationTestData(features: Vector, 
label: Int)
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  test("params") {
+ParamsSuite.checkParams(new ClusteringEvaluator)
+  }
+
+  test("read/write") {
+val evaluator = new ClusteringEvaluator()
+  .setPredictionCol("myPrediction")
+  .setFeaturesCol("myLabel")
+testDefaultReadWrite(evaluator)
+  }
+
+  /*
+Use the following python code to load the data and evaluate it using 
scikit-learn package.
+
+from sklearn import datasets
+from sklearn.metrics import silhouette_score
+iris = datasets.load_iris()
+round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 
10)
+
+0.6564679231
+  */
+  test("squared euclidean Silhouette") {
+val result = BigDecimal(0.6564679231)
+val iris = ClusteringEvaluatorSuite.irisDataset(spark)
+val evaluator = new ClusteringEvaluator()
+.setFeaturesCol("features")
+.setPredictionCol("label")
+val actual = BigDecimal(evaluator.evaluate(iris))
+  .setScale(10, BigDecimal.RoundingMode.HALF_UP)
+
+assertResult(result)(actual)
+  }
+
+}
+
+object ClusteringEvaluatorSuite {
+  def irisDataset(spark: SparkSession): DataFrame = {
+import spark.implicits._
+
+val irisCsvPath = Thread.currentThread()
+  .getContextClassLoader
+  .getResource("test-data/iris.csv")
+  .toString
--- End diff --

So this testsuite reference another testdata file. Can we generate the 
testdata in the code? Like other testsuite.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-18 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133961918
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-18 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133958240
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]
  

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-17 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133876325
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-17 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133875990
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
--- End diff --

Usually we should paste the formula here to explain how we compute 
```Silhouette Coefficient``` by the high efficient distributed implementation. 
Because your design document is not a publication, so I think we need to move 
it from there, but you can simplify it. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-17 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133747305
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]
  

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-17 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133745930
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
--- End diff --

I included the link to the design document here: 
https://github.com/mgaido91/spark/blob/ffc17f929dd86d1e7e73931eac5663bc08b6ba7a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala#L37.
 Should I move it from there? Or should I rewrite the content of the document 
in an annotation here? Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-17 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133744052
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
--- End diff --

I can't see in the wiki any of the other evaluators. And I don't see a 
detailed explanation of the maths behind the algorithms either. Thus I am not 
sure it is the best place.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133571846
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
--- End diff --

Yes, the idea often crosses my mind.
Even though there's a claim that [K-Means is for Euclidean distances 
only](https://stats.stackexchange.com/questions/81481/why-does-k-means-clustering-algorithm-use-only-euclidean-distance-metric),
 I often see people has the requirement for custom distance computation in 
practice. So I would like to see KMeans support it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133476670
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
--- End diff --

@mgaido91 Sorry I mistakenly thought to put it in the src resource rather 
than test resource. Usually we generate some dataset to verify MLlib result, we 
never put existing dataset in resource even test scope until now, but the iris 
dataset is so popular and can be used to verify lots of algorithms, so I'm OK 
to put it there. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133385546
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
--- End diff --

Sorry but I can't understand your point. Resources in the test scope are 
not included in the compiled jars. The same approach is used in the `sql` 
component for instance, where the test data is in the resources 
(https://github.com/apache/spark/tree/master/sql/core/src/test/resources/test-data).
If I generate randomly test data, I have to first perform a clustering on 
those points, while with this dataset I have the result of the clustering ready 
too. I am not sure this is the best approach. But maybe I am missing something. 
Can you please clarify this to me?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133368243
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
--- End diff --

It maybe better to refer to the wiki and explain your method in the 
`ml-clustering.md`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133370279
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133372318
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
--- End diff --

@mgaido91 You can set seed to control the randomly generated data.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133368511
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
--- End diff --

If only Euclidean is support for now, here `val metric` and `match` are not 
needed, directly return `SquaredEuclideanSilhouette.computeSquaredSilhouette...`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133372968
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
+  Row(Vectors.dense(4.9, 3.0, 1.4, 0.2), 0),
+  Row(Vectors.dense(4.7, 3.2, 1.3, 0.2), 0),
+  Row(Vectors.dense(4.6, 3.1, 1.5, 0.2), 0),
+  Row(Vectors.dense(5.0, 3.6, 1.4, 0.2), 0),
+  Row(Vectors.dense(5.4, 3.9, 1.7, 0.4), 0),
+  Row(Vectors.dense(4.6, 3.4, 1.4, 0.3), 0),
+  Row(Vectors.dense(5.0, 3.4, 1.5, 0.2), 0),
+  Row(Vectors.dense(4.4, 2.9, 1.4, 0.2), 0),
+  Row(Vectors.dense(4.9, 3.1, 1.5, 0.1), 0),
+  Row(Vectors.dense(5.4, 3.7, 1.5, 0.2), 0),
+  Row(Vectors.dense(4.8, 3.4, 1.6, 0.2), 0),
+  Row(Vectors.dense(4.8, 3.0, 1.4, 0.1), 0),
+  Row(Vectors.dense(4.3, 3.0, 1.1, 0.1), 0),
+  Row(Vectors.dense(5.8, 4.0, 1.2, 0.2), 0),
+  Row(Vectors.dense(5.7, 4.4, 1.5, 0.4), 0),
+  Row(Vectors.dense(5.4, 3.9, 1.3, 0.4), 0),
+  Row(Vectors.dense(5.1, 3.5, 1.4, 0.3), 0),
+  Row(Vectors.dense(5.7, 3.8, 1.7, 0.3), 0),
+  Row(Vectors.dense(5.1, 3.8, 1.5, 0.3), 0),
+  Row(Vectors.dense(5.4, 3.4, 1.7, 0.2), 0),
+  Row(Vectors.dense(5.1, 3.7, 1.5, 0.4), 0),
+  Row(Vectors.dense(4.6, 3.6, 1.0, 0.2), 0),
+  Row(Vectors.dense(5.1, 3.3, 1.7, 0.5), 0),
+  Row(Vectors.dense(4.8, 3.4, 1.9, 0.2), 0),
+  Row(Vectors.dense(5.0, 3.0, 1.6, 0.2), 0),
+  Row(Vectors.dense(5.0, 3.4, 1.6, 0.4), 0),
+  Row(Vectors.dense(5.2, 3.5, 1.5, 0.2), 0),
+  Row(Vectors.dense(5.2, 3.4, 1.4, 0.2), 0),
+  Row(Vectors.dense(4.7, 3.2, 1.6, 0.2), 0),
+  Row(Vectors.dense(4.8, 3.1, 1.6, 0.2), 0),
+  Row(Vectors.dense(5.4, 3.4, 1.5, 0.4), 0),
+  Row(Vectors.dense(5.2, 4.1, 1.5, 0.1), 0),
+  Row(Vectors.dense(5.5, 4.2, 1.4, 0.2), 0),
+  Row(Vectors.dense(4.9, 3.1, 1.5, 0.1), 0),
+  Row(Vectors.dense(5.0, 3.2, 1.2, 0.2), 0),
+  Row(Vectors.dense(5.5, 3.5, 1.3, 0.2), 0),
+  Row(Vectors.dense(4.9, 3.1, 1.5, 0.1), 0),
+  Row(Vectors.dense(4.4, 3.0, 1.3, 0.2), 0),
+  Row(Vectors.dense(5.1, 3.4, 1.5, 0.2), 0),
+  Row(Vectors.dense(5.0, 3.5, 1.3, 0.3), 0),
+  Row(Vectors.dense(4.5, 2.3, 1.3, 0.3), 0),
+  Row(Vectors.dense(4.4, 3.2, 1.3, 0.2), 0),
+  Row(Vectors.dense(5.0, 3.5, 1.6, 0.6), 0),
+  Row(Vectors.dense(5.1, 3.8, 1.9, 0.4), 0),
+  Row(Vectors.dense(4.8, 3.0, 1.4, 0.3), 0),
+  Row(Vectors.dense(5.1, 3.8, 1.6, 0.2), 0),
+  Row(Vectors.dense(4.6, 3.2, 1.4, 0.2), 0),
+  Row(Vectors.dense(5.3, 3.7, 1.5, 0.2), 0),
+  Row(Vectors.dense(5.0, 3.3, 1.4, 0.2), 0),
+  Row(Vectors.dense(7.0, 3.2, 4.7, 1.4), 1),
+  Row(Vectors.dense(6.4, 3.2, 4.5, 1.5), 1),
+  Row(Vectors.dense(6.9, 3.1, 4.9, 1.5), 1),
+  Row(Vectors.dense(5.5, 2.3, 4.0, 1.3), 1),
+  Row(Vectors.dense(6.5, 2.8, 4.6, 1.5), 1),
+  Row(Vectors.dense(5.7, 2.8, 4.5, 1.3), 1),
+  Row(Vectors.dense(6.3, 3.3, 4.7, 1.6), 1),
+  Row(Vectors.dense(4.9, 2.4, 3.3, 1.0), 1),
+  Row(Vectors.dense(6.6, 2.9, 4.6, 1.3), 1),
+  Row(Vectors.dense(5.2, 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133370197
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133372183
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133371918
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-16 Thread zhengruifeng
Github user zhengruifeng commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133370353
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
--- End diff --

remove empty line, and otherwise


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133360674
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
--- End diff --

I think we can't put test data in resource file, as resource file will be 
packaged in the final jar file. What about randomly generated some small data 
in Python and hard code them here? Just like what we did in 
[```GaussianMixtureSuite``` 
](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala#L195).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133360284
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
--- End diff --

Yeah, I think we can add a new param for the distance metric in the future. 
As MLlib only support _squared Euclidean distance_ , we can ignore this param 
and add annotation in the API to clarify it currently. You can check MLlib 
```KMeans```, there is no param to set distance metric. cc @jkbradley @MLnick 
@hhbyyh 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133185455
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
--- End diff --

I forgot to remove this line, I am doing it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133185265
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
--- End diff --

Unfortunately {{KMeansSuite}} and {{GaussianMixtureSuite}} use randomly 
generated data: thus it is not possible to know which should be the output 
value for the Silhouette in advance. What if I move the data to a resource file 
and read it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133182964
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
--- End diff --

Should I introduce then a new param for the distance metric? I think it is 
important to highlight that the used distance measure is the squared Euclidean 
distance, because anybody would assume that the Euclidean distance is used, if 
we don't specify it very well IMHO.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133159218
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133175278
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133159306
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133176185
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
--- End diff --

Add ```@Since("2.3.0")``` here and other places necessary.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133158750
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133157997
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133157575
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133175654
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133177805
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
--- End diff --

It's better to have some annotation to explain how we compute ```Silhouette 
Coefficient``` by the high efficient distributed implementation. You can refer 
what we did at 
[LogisticRegression](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala#L60).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the 

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133158077
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133175365
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133157195
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
+new Param(
+  this,
+  "metricName",
+  "metric name in evaluation (squaredSilhouette)",
+  allowedParams
+)
+  }
+
+  /** @group getParam */
+  def getMetricName: String = $(metricName)
+
+  /** @group setParam */
+  def setMetricName(value: String): this.type = set(metricName, value)
+
+  setDefault(metricName -> "squaredSilhouette")
+
+  override def evaluate(dataset: Dataset[_]): Double = {
+SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new 
VectorUDT)
+SchemaUtils.checkColumnType(dataset.schema, $(predictionCol), 
IntegerType)
+
+val metric: Double = $(metricName) match {
+  case "squaredSilhouette" =>
+SquaredEuclideanSilhouette.computeSquaredSilhouette(
+  dataset,
+  $(predictionCol),
+  $(featuresCol)
+)
+}
+metric
+  }
+
+}
+
+
+object ClusteringEvaluator
+  extends DefaultParamsReadable[ClusteringEvaluator] {
+
+  override def load(path: String): ClusteringEvaluator = super.load(path)
+
+}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]

[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133176770
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
+
+  override def copy(pMap: ParamMap): ClusteringEvaluator = 
this.defaultCopy(pMap)
+
+  override def isLargerBetter: Boolean = true
+
+  /** @group setParam */
+  def setPredictionCol(value: String): this.type = set(predictionCol, 
value)
+
+  /** @group setParam */
+  def setFeaturesCol(value: String): this.type = set(featuresCol, value)
+
+  /**
+   * param for metric name in evaluation
+   * (supports `"squaredSilhouette"` (default))
+   * @group param
+   */
+  val metricName: Param[String] = {
+val allowedParams = ParamValidators.inArray(Array("squaredSilhouette"))
--- End diff --

```squaredSilhouette``` -> ```silhouette```? If we support other distance 
like cosine, the metric name should be the same. The distance metric should be 
controlled by other param.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133178531
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+
+class ClusteringEvaluatorSuite
+  extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  import testImplicits._
+
+  val dataset = Seq(Row(Vectors.dense(5.1, 3.5, 1.4, 0.2), 0),
--- End diff --

It's good to have this to verify the correctness of your implementation, 
but usually we don't hard code so much data for test. Could you try to find 
existing data in ```KMeansSuite``` or ```GaussianMixtureSuite``` for testing? 
If the hard code is necessary, please try to use small dataset.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-15 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r133176352
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -0,0 +1,240 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, 
VectorUDT}
+import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators}
+import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol}
+import org.apache.spark.ml.util.{DefaultParamsReadable, 
DefaultParamsWritable, Identifiable, SchemaUtils}
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.functions.{avg, col, udf}
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Evaluator for clustering results.
+ * At the moment, the supported metrics are:
+ *  squaredSilhouette: silhouette measure using the squared Euclidean 
distance;
+ *  cosineSilhouette: silhouette measure using the cosine distance.
+ *  The implementation follows the proposal explained
+ * https://drive.google.com/file/d/0B0Hyo%5f%5fbG%5f3fdkNvSVNYX2E3ZU0/view;>
+ *   in this document.
+ */
+@Experimental
+class ClusteringEvaluator (val uid: String)
+  extends Evaluator with HasPredictionCol with HasFeaturesCol with 
DefaultParamsWritable {
+
+  def this() = this(Identifiable.randomUID("SquaredEuclideanSilhouette"))
--- End diff --

```SquaredEuclideanSilhouette``` -> ```cluEval```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-08 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r131891038
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/SquaredEuclideanSilhouette.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{Vector, VectorElementWiseSum}
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.functions.{col, count, sum}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]
+)
+  )
+  kryoRegistrationPerformed = true
+}
+  }
+
+  case class ClusterStats(Y: Vector, psi: Double, count: Long)
+
+  def computeCsi(vector: Vector): Double = {
+var sumOfSquares = 0.0
+vector.foreachActive((_, v) => {
+  sumOfSquares += v * v
+})
+sumOfSquares
+  }
+
+  def computeYVectorPsiAndCount(
+  df: DataFrame,
+  predictionCol: String,
+  featuresCol: String): DataFrame = {
+val Yudaf = new VectorElementWiseSum()
+df.groupBy(predictionCol)
+  .agg(
+count("*").alias("count"),
+sum("csi").alias("psi"),
+Yudaf(col(featuresCol)).alias("y")
--- End diff --

Please rename ```csi``` to ```squaredNorm```, ```psi``` to 
```squaredNormSum```, ```y``` to ```featureSum``` if I don't have 
misunderstand. We should use more descriptive name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18538: [SPARK-14516][ML] Adding ClusteringEvaluator with...

2017-08-08 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/18538#discussion_r131890318
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/SquaredEuclideanSilhouette.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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.evaluation
+
+import org.apache.spark.SparkContext
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.ml.linalg.{Vector, VectorElementWiseSum}
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.functions.{col, count, sum}
+
+private[evaluation] object SquaredEuclideanSilhouette {
+
+  private[this] var kryoRegistrationPerformed: Boolean = false
+
+  /**
+   * This method registers the class
+   * 
[[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]]
+   * for kryo serialization.
+   *
+   * @param sc `SparkContext` to be used
+   */
+  def registerKryoClasses(sc: SparkContext): Unit = {
+if (! kryoRegistrationPerformed) {
+  sc.getConf.registerKryoClasses(
+Array(
+  classOf[SquaredEuclideanSilhouette.ClusterStats]
+)
+  )
+  kryoRegistrationPerformed = true
+}
+  }
+
+  case class ClusterStats(Y: Vector, psi: Double, count: Long)
+
+  def computeCsi(vector: Vector): Double = {
--- End diff --

Can we use ```Vectors.norm(vector, 2.0)```? It should be more efficient for 
both dense and sparse vector. Actually we can remove this function if you 
refactor code as my suggested below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



  1   2   >