[GitHub] spark pull request: [SPARK-2236][SQL]SparkSQL add SkewJoin

2014-07-03 Thread YanjieGao
Github user YanjieGao commented on the pull request:

https://github.com/apache/spark/pull/1134#issuecomment-47870392
  
Hi all. I have resolve the conflict.


---
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: FIX: ShuffledDStream run tasks only when dstre...

2014-07-03 Thread guowei2
Github user guowei2 commented on a diff in the pull request:

https://github.com/apache/spark/pull/1291#discussion_r14498644
  
--- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
 ---
@@ -39,8 +39,10 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: 
ClassTag](
 
   override def compute(validTime: Time): Option[RDD[(K,C)]] = {
 parent.getOrCompute(validTime) match {
-  case Some(rdd) = Some(rdd.combineByKey[C](
-  createCombiner, mergeValue, mergeCombiner, partitioner, 
mapSideCombine))
+  case Some(rdd) = {
+Some(if (rdd.partitions.length==0) 
rdd.combineByKey(createCombiner, mergeValue, mergeCombiner,0)
--- End diff --

1、for example:
u use NetworkInputDStream to receive data to generate BlockRDD.  when it 
receive no blocks in validtime, then partitions will be 0

NetworkInputDStream's code here 

  override def compute(validTime: Time): Option[RDD[T]] = {
// If this is called for any time before the start time of the context,
// then this returns an empty RDD. This may happen when recovering from 
a
// master failure
if (validTime = graph.startTime) {
  val blockIds = ssc.scheduler.networkInputTracker.getBlockIds(id, 
validTime)
  Some(new BlockRDD[T](ssc.sc, blockIds))
} else {
  Some(new BlockRDD[T](ssc.sc, Array[BlockId]()))
}
  }

2.  when parent RDD partition num  is 0 ,but  ShuffledDStream also change 
into muti-partitions ,
then emtpy job will be running.
  
  what i do is when this happen ShuffledDStream also out 0 partition


---
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: SPARK-1627: Support external aggregation by us...

2014-07-03 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/867#issuecomment-47871283
  
This is very helpful when the partition can not fit in memory. However, I 
think we'd better keep previous implementation; Aggregator in spark is for very 
typical scenario, but we do have many algorithms / ways to optimize the 
aggregation in SQL for performance, hence tightly coupled with Spark Aggregator 
may not a good idea for further improvement.



---
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: [SPARK-2082] stratified sampling in PairRDDFun...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1025#discussion_r14500775
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala 
---
@@ -27,8 +27,12 @@ import scala.collection.Map
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 import scala.reflect.ClassTag
+import scala.util.control.Breaks._
--- End diff --

This is not a native feature of Scala, neither a for loop. Try to use while 
with a flag that terminates the loop.


---
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: [SPARK-2082] stratified sampling in PairRDDFun...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1025#discussion_r14500845
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala 
---
@@ -46,6 +48,8 @@ import org.apache.spark.Partitioner.defaultPartitioner
 import org.apache.spark.SparkContext._
 import org.apache.spark.partial.{BoundedDouble, PartialResult}
 import org.apache.spark.serializer.Serializer
+import org.apache.spark.util.Utils
+import org.apache.spark.util.random.{Stratum, Result, StratifiedSampler, 
PoissonBounds = PB}
--- End diff --

Re-organize the imports (using an intellij plugin: 
http://plugins.jetbrains.com/plugin/7350?pr=)


---
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: SPARK-1782: svd for sparse matrix using ARPACK

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/964#issuecomment-47878720
  
@yangliuyu What did you set for `k` and how many iterations it took?


---
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: SPARK-1782: svd for sparse matrix using ARPACK

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/964#issuecomment-47881287
  
@vrilleup Both approaches compute the truncated SVD. I still prefer putting 
both implementation under `computeSVD` for now. I'm going to implement a 
generic ParameterSet (similar to Matlab's `options`). Once we have that, we can 
add it to this method and users can choose which to use by setting `method` to 
`dense`, `arpack`, or `auto`.

It is okay to use a simple rule to decide which approach to take. For 
example, if `n  100` or `k  n / 2` use dense, otherwise, use sparse. For the 
dense approach, we can also use ARPACK instead of full SVD to compute the top k 
singular values of the gram matrix. Just put a TODO to the condition and say 
this is not optimized.  


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504837
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
--- End diff --

Please mark new methods @Experimental.


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504879
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
--- End diff --

ditto


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504877
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
--- End diff --

Recall - recall


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504864
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
--- End diff --

ditto


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504858
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
--- End diff --

same style issue


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504861
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
--- End diff --

Precision - precision


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504853
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
--- End diff --

Please follow [Spark Code Style 
Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide)
 and code style used in the code base.

~~~
.map { case (prediction, label) =
  (label, if (label == prediction) 1 else 0)
}.reduceByKey(_ + _)
.collectAsMap()
~~~

1) new line after  =
2) space after if
3) change { } to ( )
4) use () for an action


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504868
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
--- End diff --

Make a closure for this function and make the code align better.

~~~
... : Double = {
  val tp = tpByClass(label)
  val fp = fpByClass.getOrElse(label, 0)
  if (tp + fp == 0) 0 else tp.toDouble / (tp + fp)
}
~~~


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504845
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
--- End diff --

What is this comment for?


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504893
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
--- End diff --

Could you change the method to `fMeasure(label: Double)` and 
`fMeasure(label: Double, beta: Double)`? The former computes F1 while the 
latter computes F_beta.

`={` - `= {`


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504849
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
--- End diff --

Change it to `labelCountByClass.values.sum`.


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504899
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
--- End diff --

space after if

`(p + r) == 0` - `p + r == 0`


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504884
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
--- End diff --

`tpByClass(label).toDouble / labelCountByClass(label)` (the second toDouble 
is not 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.
---


[GitHub] spark pull request: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504933
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
+  }
+
+  /**
+   * Returns micro-averaged Recall
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microRecall.
+   */
+  lazy val microRecall: Double =
+tpByClass.foldLeft(0L){case (sum,(_, tp)) = sum + tp}.toDouble / 
labelCount
--- End diff --

`tpByClass.values.sum / labelCount`


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504942
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
+  }
+
+  /**
+   * Returns micro-averaged Recall
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microRecall.
+   */
+  lazy val microRecall: Double =
+tpByClass.foldLeft(0L){case (sum,(_, tp)) = sum + tp}.toDouble / 
labelCount
+
+  /**
+   * Returns micro-averaged Precision
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microPrecision.
+   */
+  lazy val microPrecision: Double = microRecall
+
+  /**
+   * Returns micro-averaged F1-measure
+   * (equals to microPrecision and microRecall for multiclass classifier)
+   * @return microF1measure.
+   */
+  lazy val microF1Measure: Double = microRecall
--- End diff --

remove


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504938
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
+  }
+
+  /**
+   * Returns micro-averaged Recall
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microRecall.
+   */
+  lazy val microRecall: Double =
+tpByClass.foldLeft(0L){case (sum,(_, tp)) = sum + tp}.toDouble / 
labelCount
+
+  /**
+   * Returns micro-averaged Precision
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microPrecision.
+   */
+  lazy val microPrecision: Double = microRecall
--- End diff --

remove


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504928
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
+  }
+
+  /**
+   * Returns micro-averaged Recall
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microRecall.
+   */
+  lazy val microRecall: Double =
--- End diff --

This is not useful. It gives you the global precision and the method name 
micro is confusing. We can simply call it `precision()` and  remove `micro*` 
methods.


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504960
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
+  }
+
+  /**
+   * Returns micro-averaged Recall
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microRecall.
+   */
+  lazy val microRecall: Double =
+tpByClass.foldLeft(0L){case (sum,(_, tp)) = sum + tp}.toDouble / 
labelCount
+
+  /**
+   * Returns micro-averaged Precision
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microPrecision.
+   */
+  lazy val microPrecision: Double = microRecall
+
+  /**
+   * Returns micro-averaged F1-measure
+   * (equals to microPrecision and microRecall for multiclass classifier)
+   * @return microF1measure.
+   */
+  lazy val microF1Measure: Double = microRecall
+
+  /**
+   * Returns weighted averaged Recall
+   * @return weightedRecall.
+   */
+  lazy val weightedRecall: Double = 
labelCountByClass.foldLeft(0.0){case(wRecall, (category, count)) =
--- End diff --

This looks better to me:

~~~
weightedRecall: Double = labelCountByClass.map { case (category, count) =
recall(category) * count / labelCount
  }.sum
~~~


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14504967
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala 
---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+
+/**
+ * Evaluator for multiclass classification.
+ * NB: type Double both for prediction and label is retained
+ * for compatibility with model.predict that returns Double
+ * and MLUtils.loadLibSVMFile that loads class labels as Double
+ *
+ * @param predictionsAndLabels an RDD of (prediction, label) pairs.
+ */
+class MulticlassMetrics(predictionsAndLabels: RDD[(Double, Double)]) 
extends Logging {
+
+  /* class = category; label = instance of class; prediction = instance of 
class */
+
+  private lazy val labelCountByClass = 
predictionsAndLabels.values.countByValue()
+  private lazy val labelCount = labelCountByClass.foldLeft(0L){case(sum, 
(_, count)) = sum + count}
+  private lazy val tpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(label, if(label == prediction) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+  private lazy val fpByClass = predictionsAndLabels.map{ case (prediction, 
label) =
+(prediction, if(prediction != label) 1 else 0) }.reduceByKey{_ + 
_}.collectAsMap
+
+  /**
+   * Returns Precision for a given label (category)
+   * @param label the label.
+   * @return Precision.
+   */
+  def precision(label: Double): Double = if(tpByClass(label) + 
fpByClass.getOrElse(label, 0) == 0) 0
+else tpByClass(label).toDouble / (tpByClass(label) + 
fpByClass.getOrElse(label, 0)).toDouble
+
+  /**
+   * Returns Recall for a given label (category)
+   * @param label the label.
+   * @return Recall.
+   */
+  def recall(label: Double): Double = tpByClass(label).toDouble / 
labelCountByClass(label).toDouble
+
+  /**
+   * Returns F1-measure for a given label (category)
+   * @param label the label.
+   * @return F1-measure.
+   */
+  def f1Measure(label: Double): Double ={
+val p = precision(label)
+val r = recall(label)
+if((p + r) == 0) 0 else 2 * p * r / (p + r)
+  }
+
+  /**
+   * Returns micro-averaged Recall
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microRecall.
+   */
+  lazy val microRecall: Double =
+tpByClass.foldLeft(0L){case (sum,(_, tp)) = sum + tp}.toDouble / 
labelCount
+
+  /**
+   * Returns micro-averaged Precision
+   * (equals to microPrecision and microF1measure for multiclass 
classifier)
+   * @return microPrecision.
+   */
+  lazy val microPrecision: Double = microRecall
+
+  /**
+   * Returns micro-averaged F1-measure
+   * (equals to microPrecision and microRecall for multiclass classifier)
+   * @return microF1measure.
+   */
+  lazy val microF1Measure: Double = microRecall
+
+  /**
+   * Returns weighted averaged Recall
+   * @return weightedRecall.
+   */
+  lazy val weightedRecall: Double = 
labelCountByClass.foldLeft(0.0){case(wRecall, (category, count)) =
+wRecall + recall(category) * count.toDouble / labelCount}
+
+  /**
+   * Returns weighted averaged Precision
+   * @return weightedPrecision.
+   */
+  lazy val weightedPrecision: Double =
+labelCountByClass.foldLeft(0.0){case(wPrecision, (category, count)) =
+wPrecision + precision(category) * count.toDouble / labelCount}
+
+  /**
+   * Returns weighted averaged F1-measure
+   * @return weightedF1Measure.
+   */
+  lazy val weightedF1Measure: Double =
+labelCountByClass.foldLeft(0.0){case(wF1measure, (category, count)) =
+wF1measure + f1Measure(category) * 

[GitHub] spark pull request: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14505018
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala
 ---
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.mllib.util.LocalSparkContext
+import org.scalatest.FunSuite
--- End diff --

organize imports into groups


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14505019
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala
 ---
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.mllib.util.LocalSparkContext
+import org.scalatest.FunSuite
+
+class MulticlassMetricsSuite extends FunSuite with LocalSparkContext {
+  test(Multiclass evaluation metrics) {
+/*
+* Confusion matrix for 3-class classification with total 9 instances:
+* |2|1|1| true class0 (4 instances)
+* |1|3|0| true class1 (4 instances)
+* |0|0|1| true class2 (1 instance)
+*
+*/
+val scoreAndLabels = sc.parallelize(
+  Seq((0.0, 0.0), (0.0, 1.0), (0.0, 0.0), (1.0, 0.0), (1.0, 1.0),
+(1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2)
+val metrics = new MulticlassMetrics(scoreAndLabels)
+
+val delta = 0.1
--- End diff --

Use a smaller delta.


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/1155#discussion_r14505016
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala
 ---
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.evaluation
+
+import org.apache.spark.mllib.util.LocalSparkContext
+import org.scalatest.FunSuite
+
+class MulticlassMetricsSuite extends FunSuite with LocalSparkContext {
+  test(Multiclass evaluation metrics) {
+/*
+* Confusion matrix for 3-class classification with total 9 instances:
+* |2|1|1| true class0 (4 instances)
+* |1|3|0| true class1 (4 instances)
+* |0|0|1| true class2 (1 instance)
+*
+*/
+val scoreAndLabels = sc.parallelize(
+  Seq((0.0, 0.0), (0.0, 1.0), (0.0, 0.0), (1.0, 0.0), (1.0, 1.0),
+(1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2)
+val metrics = new MulticlassMetrics(scoreAndLabels)
+
+val delta = 0.1
+val precision0 = 2.0 / (2.0 + 1.0)
+val precision1 = 3.0 / (3.0 + 1.0)
+val precision2 = 1.0 / (1.0 + 1.0)
+val recall0 = 2.0 / (2.0 + 2.0)
+val recall1 = 3.0 / (3.0 + 1.0)
+val recall2 = 1.0 / (1.0 + 0.0)
+val f1measure0 = 2 * precision0 * recall0 / (precision0 + recall0)
+val f1measure1 = 2 * precision1 * recall1 / (precision1 + recall1)
+val f1measure2 = 2 * precision2 * recall2 / (precision2 + recall2)
+
+assert(math.abs(metrics.precision(0.0) - precision0)  delta)
+assert(math.abs(metrics.precision(1.0) - precision1)  delta)
+assert(math.abs(metrics.precision(2.0) - precision2)  delta)
+assert(math.abs(metrics.recall(0.0) - recall0)  delta)
+assert(math.abs(metrics.recall(1.0) - recall1)  delta)
+assert(math.abs(metrics.recall(2.0) - recall2)  delta)
+assert(math.abs(metrics.f1Measure(0.0) - f1measure0)  delta)
+assert(math.abs(metrics.f1Measure(1.0) - f1measure1)  delta)
+assert(math.abs(metrics.f1Measure(2.0) - f1measure2)  delta)
+
+assert(math.abs(metrics.microRecall -
+  (2.0 + 3.0 + 1.0) / ((2.0 + 3.0 + 1.0) + (1.0 + 1.0 + 1.0)))  delta)
+assert(math.abs(metrics.microRecall - metrics.microPrecision)  delta)
+assert(math.abs(metrics.microRecall - metrics.microF1Measure)  delta)
+assert(math.abs(metrics.microRecall - metrics.weightedRecall)  delta)
+assert(math.abs(metrics.weightedPrecision -
+  ((4.0 / 9.0) * precision0 + (4.0 / 9.0) * precision1 + (1.0 / 9.0) * 
precision2))  delta)
+assert(math.abs(metrics.weightedRecall -
+  ((4.0 / 9.0) * recall0 + (4.0 / 9.0) * recall1 + (1.0 / 9.0) * 
recall2))  delta)
+assert(math.abs(metrics.weightedF1Measure -
+  ((4.0 / 9.0) * f1measure0 + (4.0 / 9.0) * f1measure1 + (1.0 / 9.0) * 
f1measure2))  delta)
+
--- End diff --

remove empty line


---
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: [MLLIB] [SPARK-2222] Add multiclass evaluation...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1155#issuecomment-47886626
  
@avulanov Thanks for adding metrics for multiclass classification! I made 
one pass. Besides the inline comments, how can we handle models that output raw 
scores for each category, e.g., prediction = {0: 0.05, 1: 0.11, 2: 0.88}, in 
the future?


---
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: SPARK-1782: svd for sparse matrix using ARPACK

2014-07-03 Thread yangliuyu
Github user yangliuyu commented on the pull request:

https://github.com/apache/spark/pull/964#issuecomment-47886598
  
@mengxr k is 100, rCond, tol and maxIterations are all defaults, i.e. 1e-9, 
1e-10, 300. Change iteration number from 300 to 200 will not reduce too much 
time cost 10% (test on another small size of dataset 61794 x 100, k=99 ~110s)

btw, improve RowMatrix multiply patch works well.


---
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: rename the TestRelation to LocalRelation

2014-07-03 Thread scwf
Github user scwf closed the pull request at:

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


---
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: Mllib grouped optimization

2014-07-03 Thread kellrott
GitHub user kellrott opened a pull request:

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

Mllib grouped optimization

The purpose of this patch is the enable MLLib to better handle scenarios 
where the user would want to do learning on multiple feature/label sets at the 
same time. Rather then schedule each learning task separately, this patch lets 
the user create a single RDD with an Int key to represent the 'group' sets of 
entries belong to.

This patch establishing the GroupedOptimizer trait, for which 
GroupedGradientDescent  has been implemented. This systems differs from the 
original Optimizer trait in that the original optimize method accepted 
RDD[(Int, Vector)] the new GroupedOptimizer accepts RDD[(Int, (Double, 
Vector))].
The difference is that the GroupedOptimizer uses a 'group' ID key in the 
RDD to multiplex multiple optimization operations in the same RDD.

This patch also establishes the GroupedGeneralizedLinearAlgorithm trait, 
for which the 'run' method has had the RDD[LabeledPoint] input replaced with 
RDD[(Int,LabeledPoint)].

This patch also provides a unit test and utility to take the results of 
MLUtils.kFold and turn it into a single grouped RDD, ready for simultaneous 
learning. 

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/kellrott/spark mllib-grouped

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/1292.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1292


commit 664196a78cece095ac78293379503afc9f14c2c9
Author: Kyle Ellrott kellr...@gmail.com
Date:   2014-07-01T16:45:16Z

Adding files to do grouped optimization (Gradient Decent right now)

commit f99c8abf5322ddb73bd5f56a53a4784d5f20e8cf
Author: Kyle Ellrott kellr...@gmail.com
Date:   2014-07-01T20:53:03Z

Adding GroupedGeneralizedLinearAlgorithm class

commit 02a192adb5daf8b1812bac7ad6ba0b2233040208
Author: Kyle Ellrott kellr...@gmail.com
Date:   2014-07-02T20:21:22Z

Working GroupedSVM and unit tests




---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47947046
  
 Merged build triggered. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47947062
  
Merged build started. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47947235
  
Merged build finished. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47947236
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16303/


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47948524
  
Merged build started. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47948510
  
 Merged build triggered. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47948723
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16304/


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47948720
  
Merged build finished. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47951037
  
 Merged build triggered. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47951045
  
Merged build started. 


---
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: [SPARK-2350] Don't NPE while launching drivers

2014-07-03 Thread aarondav
Github user aarondav commented on the pull request:

https://github.com/apache/spark/pull/1289#issuecomment-47957745
  
This should be merged into branch-1.0 and branch-0.9 as well, as this bug 
disallows queuing up standalone drivers.


---
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: [SPARK-2355] Add checker for the number of clu...

2014-07-03 Thread viirya
GitHub user viirya opened a pull request:

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

[SPARK-2355] Add checker for the number of clusters

When the number of clusters given to perform with 
org.apache.spark.mllib.clustering.KMeans under parallel initial mode is greater 
than data number, it will throw ArrayIndexOutOfBoundsException.

This PR adds checker for the number of clusters and throws 
IllegalArgumentException when that number is greater than data number.



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/viirya/spark-1 check_clusters_number

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/1293.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1293


commit 582cd11e5331a8e2704a5603080eec41c9002cf4
Author: Liang-Chi Hsieh vii...@gmail.com
Date:   2014-07-03T16:27:22Z

simply add checker for the number of clusters.




---
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: [SPARK-2324] SparkContext should not exit dire...

2014-07-03 Thread aarondav
Github user aarondav commented on the pull request:

https://github.com/apache/spark/pull/1274#issuecomment-47958114
  
LGTM. Merging into master.


---
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: [SPARK-2324] SparkContext should not exit dire...

2014-07-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-2355] Add checker for the number of clu...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1293#issuecomment-47958358
  
Can one of the admins verify this patch?


---
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: [SPARK-2355] Add checker for the number of clu...

2014-07-03 Thread aarondav
Github user aarondav commented on the pull request:

https://github.com/apache/spark/pull/1293#issuecomment-47959115
  
data.count() is actually a very expensive operation, as it has to scan all 
the data. If cached, it may not be as much a problem, but it is still probably 
not worth it for this check. Which part throws the exception?


---
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: [SPARK] Fix NPE for ExternalAppendOnlyMap

2014-07-03 Thread aarondav
Github user aarondav commented on the pull request:

https://github.com/apache/spark/pull/1288#issuecomment-47959481
  
LGTM, merging into master and branch-1.0.


---
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: [SPARK] Fix NPE for ExternalAppendOnlyMap

2014-07-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK] Fix NPE for ExternalAppendOnlyMap

2014-07-03 Thread aarondav
Github user aarondav commented on the pull request:

https://github.com/apache/spark/pull/1288#issuecomment-47959809
  
This fixes [SPARK-2349](https://issues.apache.org/jira/browse/SPARK-2349).


---
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: [SPARK-2350] Don't NPE while launching drivers

2014-07-03 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/1289#issuecomment-47959923
  
LGTM


---
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: [SPARK-2355] Add checker for the number of clu...

2014-07-03 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/1293#issuecomment-47961640
  
The problem lies in `initKMeansParallel`, the implementation of k-means|| 
algorithm. Since it selects at most the centers as many as the data number, 
when calling `LocalKMeans.kMeansPlusPlus` at the end of `initKMeansParallel`, 
`kMeansPlusPlus` would throw this exception.

I can slightly modify `kMeansPlusPlus` to avoid this exception by selected 
chosen centers to fill the gap between cluster numbers and data number. But 
this approach might not be appropriate because it is not the problem of the 
algorithm.

I also think about whether it is worth to check that by scanning all data. 
But since it is only counting and no other computations involved, it might be 
acceptable still. In fact, there are also many map operations on the data later 
in clustering. Comparing with these map ops, `data.count()` should be 
lightweight more? Or it is unnecessary to check that? Any suggestions?


---
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: Mllib grouped optimization

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47961794
  
@kellrott Please create a JIRA and then add both the JIRA number and 
`[MLLIB]` to the title of this PR, following 
https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark . It 
makes easier to find track the progress and locate MLlib's PRs.


---
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: [spark-2352] Implementation of an 1-hidden lay...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1290#issuecomment-47962042
  
@bgreeven Please add `[MLLIB]` to your PR, following 
https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark . It 
makes easier for people who want to search MLlib's PRs. 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.
---


[GitHub] spark pull request: [spark-2352] Implementation of an 1-hidden lay...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1290#issuecomment-47962057
  
Jenkins, test this please.


---
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: [spark-2352] Implementation of an 1-hidden lay...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1290#issuecomment-47962391
  
 Merged build triggered. 


---
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: [spark-2352] Implementation of an 1-hidden lay...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1290#issuecomment-47962409
  
Merged build started. 


---
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: [spark-2352] Implementation of an 1-hidden lay...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1290#issuecomment-47962626
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16306/


---
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: SPARK-1782: svd for sparse matrix using ARPACK

2014-07-03 Thread vrilleup
Github user vrilleup commented on the pull request:

https://github.com/apache/spark/pull/964#issuecomment-47963684
  
@mengxr the plan sounds good, I will make the changes. I had some 
experiences on designing a generic ParameterSet, including constraints on 
individual parameter, constraints across several parameters, and parameter 
sweeps (I don't think sweep is needed in milib though). Please let me know if I 
can help.


---
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: [MLLIB] SPARK-1682: Add gradient descent w/o s...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/643#issuecomment-47963757
  
 Merged build triggered. 


---
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: [MLLIB] SPARK-1682: Add gradient descent w/o s...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/643#issuecomment-47963771
  
Merged build started. 


---
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: SPARK-1782: svd for sparse matrix using ARPACK

2014-07-03 Thread vrilleup
Github user vrilleup commented on the pull request:

https://github.com/apache/spark/pull/964#issuecomment-47964502
  
@yangliuyu did you persist the underlying RDD[Vector] behind RowMatrix? 
This RDD is used in every iteration,  caching will improve run time a lot. From 
my tests, if the RDD is persist(), the first call to aggregate takes 10s to 
1min, while any subsequent aggregate takes less than 1s. 


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47965725
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16305/


---
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: Mllib grouped optimization

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1292#issuecomment-47965718
  
Merged build finished. 


---
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: [SPARK-2234][SQL]Spark SQL basicOperators add ...

2014-07-03 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/1151#discussion_r14529904
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala ---
@@ -135,13 +137,12 @@ class SqlParser extends StandardTokenParsers with 
PackratParsers {
 }
   }
 
-  protected lazy val query: Parser[LogicalPlan] = (
-select * (
-UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) = Union(q1, 
q2) } |
-UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) = 
Distinct(Union(q1, q2)) }
-  )
-| insert | cache
-  )
+  protected lazy val query: Parser[LogicalPlan] =
+   select * (
+   UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) = Union(q1, 
q2)} |
+   EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) = Except(q1, q2)} |
+   UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) = 
Distinct(Union(q1, q2))}
+   ) | insert | cache
--- End diff --

Please undo all the indentation changes to these lines, and add a space 
before all `}`s.


---
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: [SPARK-2234][SQL]Spark SQL basicOperators add ...

2014-07-03 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/1151#discussion_r14529948
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala ---
@@ -296,3 +298,5 @@ private[sql] abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
 }
   }
 }
+
--- End diff --

Remove extra new lines.


---
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: [SPARK-2234][SQL]Spark SQL basicOperators add ...

2014-07-03 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/1151#discussion_r14529947
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -371,6 +371,19 @@ class SQLQuerySuite extends QueryTest {
 (3, null)))
   }
 
+  test(EXCEPT) {
+
+checkAnswer(
+  sql(SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData 
),
+  (1, a) ::
+  (2, b) ::
+  (3, c) ::
+  (4, d) :: Nil)
+checkAnswer(
+  sql(SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData 
), Nil)
+checkAnswer(
+  sql(SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData 
), Nil)
+  }
   test(SET commands semantics using sql()) {
--- End diff --

Add a newline before this line.


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread marmbrus
Github user marmbrus commented on the pull request:

https://github.com/apache/spark/pull/1283#issuecomment-47966853
  
test this please


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1283#issuecomment-47967655
  
 Merged build triggered. 


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1283#issuecomment-47967682
  
Merged build started. 


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47968583
  
Jenkins, retest this please.


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47968908
  
Merged build started. 


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47968895
  
 Merged build triggered. 


---
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: [SPARK-1199][REPL] Remove VALId and use the or...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1179#issuecomment-47968906
  
Merged build started. 


---
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: [SPARK-1199][REPL] Remove VALId and use the or...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1179#issuecomment-47968891
  
 Merged build triggered. 


---
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: [MLLIB] SPARK-1682: Add gradient descent w/o s...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/643#issuecomment-47969923
  
Merged build finished. All automated tests passed.


---
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: [MLLIB] SPARK-1682: Add gradient descent w/o s...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/643#issuecomment-47969925
  
All automated tests passed.
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16307/


---
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: SPARK-1675. Make clear whether computePrincipa...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1171#issuecomment-47970667
  
Merged. 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.
---


[GitHub] spark pull request: SPARK-1675. Make clear whether computePrincipa...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1171#issuecomment-47970639
  
Merged. 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.
---


[GitHub] spark pull request: SPARK-1675. Make clear whether computePrincipa...

2014-07-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: Update SQLConf.scala

2014-07-03 Thread aarondav
Github user aarondav commented on the pull request:

https://github.com/apache/spark/pull/1272#issuecomment-47970749
  
@rxin wins because he says that SQLConf will become a Thread-local 
variable. This looks good, the only thing to change for thread-safety is to add 
a synchronized for getAll().


---
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: [SPARK-1097] Workaround Hadoop conf Concurrent...

2014-07-03 Thread aarondav
Github user aarondav commented on a diff in the pull request:

https://github.com/apache/spark/pull/1273#discussion_r14533232
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -141,7 +141,7 @@ class HadoopRDD[K, V](
   // local process. The local cache is accessed through 
HadoopRDD.putCachedMetadata().
   // The caching helps minimize GC, since a JobConf can contain ~10KB 
of temporary objects.
   // synchronize to prevent ConcurrentModificationException 
(Spark-1097, Hadoop-10456)
-  broadcastedConf.synchronized {
+  broadcastedConf.value.value.synchronized {
--- End diff --

Can we just use conf here and on the next line, instead of 
`broadcastedConf.value.value`? This is actually the first guy that assumes conf 
is not null, though, maybe add an assert for that.


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1283#issuecomment-47978597
  
Merged build finished. All automated tests passed.


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47978598
  
Merged build finished. 


---
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: [SPARK-1199][REPL] Remove VALId and use the or...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1179#issuecomment-47978599
  
All automated tests passed.
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16309/


---
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: [SPARK-1199][REPL] Remove VALId and use the or...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1179#issuecomment-47978596
  
Merged build finished. All automated tests passed.


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1283#issuecomment-47978600
  
All automated tests passed.
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16308/


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47978602
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16310/


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-2342] Evaluation helper's output type d...

2014-07-03 Thread marmbrus
Github user marmbrus commented on the pull request:

https://github.com/apache/spark/pull/1283#issuecomment-47980908
  
Thanks, I merged this into master and 1.0!


---
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: [SPARK-2059][SQL] Add analysis checks

2014-07-03 Thread liancheng
Github user liancheng commented on the pull request:

https://github.com/apache/spark/pull/1265#issuecomment-47981028
  
`HiveComparisonTest` simply ignores all `ExplainCommand` and don't check 
their output. We should catch the exception in `execution.ExplainCommand` and 
report the exception stacktrace as part of the  plan explanation rather than 
throw it out.


---
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: [SPARK-1776] Have Spark's SBT build read depen...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/772#issuecomment-47981107
  
 Merged build triggered. 


---
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: [SPARK-1776] Have Spark's SBT build read depen...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/772#issuecomment-47981128
  
Merged build started. 


---
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: [SPARK-1776] Have Spark's SBT build read depen...

2014-07-03 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/772#issuecomment-47981340
  
@ScrapCodes thanks for updating this!


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47982406
  
Jenkins, retest this please.


---
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: [SPARK-1776] Have Spark's SBT build read depen...

2014-07-03 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/772#discussion_r14536931
  
--- Diff: project/SparkBuild.scala ---
@@ -434,250 +235,41 @@ object SparkBuild extends Build {
   -noqualifier, java.lang
 )
   )
+}
 
-  def replSettings = sharedSettings ++ Seq(
-name := spark-repl,
-libraryDependencies += scalaVersion(v = org.scala-lang  % 
scala-compiler % v),
-libraryDependencies += scalaVersion(v = org.scala-lang  % jline  
% v),
-libraryDependencies += scalaVersion(v = org.scala-lang  % 
scala-reflect  % v)
-  )
-
-  def examplesSettings = sharedSettings ++ Seq(
-name := spark-examples,
-jarName in assembly = version map {
-  v = spark-examples- + v + -hadoop + hadoopVersion + .jar },
-libraryDependencies ++= Seq(
-  com.twitter  %% algebird-core   % 0.1.11,
-  org.apache.hbase % hbase % HBASE_VERSION 
excludeAll(excludeIONetty, excludeJBossNetty, excludeAsm, excludeOldAsm, 
excludeCommonsLogging, excludeJruby),
-  org.apache.cassandra % cassandra-all % 1.2.6
-exclude(com.google.guava, guava)
-exclude(com.googlecode.concurrentlinkedhashmap, 
concurrentlinkedhashmap-lru)
-exclude(com.ning,compress-lzf)
-exclude(io.netty, netty)
-exclude(jline,jline)
-exclude(org.apache.cassandra.deps, avro)
-excludeAll(excludeSLF4J, excludeIONetty),
-  com.github.scopt %% scopt % 3.2.0
-)
-  ) ++ assemblySettings ++ extraAssemblySettings
-
-  def toolsSettings = sharedSettings ++ Seq(
-name := spark-tools,
-libraryDependencies += scalaVersion(v = org.scala-lang  % 
scala-compiler % v),
-libraryDependencies += scalaVersion(v = org.scala-lang  % 
scala-reflect  % v )
-  ) ++ assemblySettings ++ extraAssemblySettings
-
-  def graphxSettings = sharedSettings ++ Seq(
-name := spark-graphx,
-previousArtifact := sparkPreviousArtifact(spark-graphx),
-libraryDependencies ++= Seq(
-  org.jblas % jblas % jblasVersion
-)
-  )
-
-  def bagelSettings = sharedSettings ++ Seq(
-name := spark-bagel,
-previousArtifact := sparkPreviousArtifact(spark-bagel)
-  )
-
-  def mllibSettings = sharedSettings ++ Seq(
-name := spark-mllib,
-previousArtifact := sparkPreviousArtifact(spark-mllib),
-libraryDependencies ++= Seq(
-  org.jblas % jblas % jblasVersion,
-  org.scalanlp %% breeze % 0.7 excludeAll(excludeJUnit)
-)
-  )
-
-  def catalystSettings = sharedSettings ++ Seq(
-name := catalyst,
-// The mechanics of rewriting expression ids to compare trees in some 
test cases makes
-// assumptions about the the expression ids being contiguous.  Running 
tests in parallel breaks
-// this non-deterministically.  TODO: FIX THIS.
-parallelExecution in Test := false,
-libraryDependencies ++= Seq(
-  com.typesafe %% scalalogging-slf4j % 1.0.1
-)
-  )
+object TestSettings {
+  import BuildCommons._
 
-  def sqlCoreSettings = sharedSettings ++ Seq(
-name := spark-sql,
-libraryDependencies ++= Seq(
-  com.twitter  % parquet-column % 
parquetVersion,
-  com.twitter  % parquet-hadoop % 
parquetVersion,
-  com.fasterxml.jackson.core   % jackson-databind   % 
2.3.0 // json4s-jackson 3.2.6 requires jackson-databind 2.3.0.
-),
-initialCommands in console :=
-  
-|import org.apache.spark.sql.catalyst.analysis._
-|import org.apache.spark.sql.catalyst.dsl._
-|import org.apache.spark.sql.catalyst.errors._
-|import org.apache.spark.sql.catalyst.expressions._
-|import org.apache.spark.sql.catalyst.plans.logical._
-|import org.apache.spark.sql.catalyst.rules._
-|import org.apache.spark.sql.catalyst.types._
-|import org.apache.spark.sql.catalyst.util._
-|import org.apache.spark.sql.execution
-|import org.apache.spark.sql.test.TestSQLContext._
-|import org.apache.spark.sql.parquet.ParquetTestData.stripMargin
-  )
+  lazy val settings = Seq (
+// Fork new JVMs for tests and set Java options for those
+fork := true,
+javaOptions in Test += -Dspark.home= + sparkHome,
+javaOptions in Test += -Dspark.testing=1,
+javaOptions in Test += -Dsun.io.serialization.extendedDebugInfo=true,
+javaOptions in Test ++= System.getProperties.filter(_._1 startsWith 
spark).map { case (k,v) = s-D$k=$v }.toSeq,
+javaOptions in Test ++= -Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m 
-XX:MaxPermSize=1g.split( 

[GitHub] spark pull request: [SPARK-1776] Have Spark's SBT build read depen...

2014-07-03 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/772#discussion_r14536966
  
--- Diff: project/SparkBuild.scala ---
@@ -15,393 +15,194 @@
  * limitations under the License.
  */
 
+import scala.util.Properties
+import scala.collection.JavaConversions._
+
 import sbt._
-import sbt.Classpaths.publishTask
 import sbt.Keys._
-import sbtassembly.Plugin._
-import AssemblyKeys._
-import scala.util.Properties
 import org.scalastyle.sbt.ScalastylePlugin.{Settings = ScalaStyleSettings}
-import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact
-import sbtunidoc.Plugin._
-import UnidocKeys._
+import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys}
+import net.virtualvoid.sbt.graph.Plugin.graphSettings
 
-import scala.collection.JavaConversions._
+object BuildCommons {
 
-// For Sonatype publishing
-// import com.jsuereth.pgp.sbtplugin.PgpKeys._
+  private val buildLocation = file(.).getAbsoluteFile.getParentFile
 
-object SparkBuild extends Build {
-  val SPARK_VERSION = 1.1.0-SNAPSHOT
-  val SPARK_VERSION_SHORT = SPARK_VERSION.replaceAll(-SNAPSHOT, )
+  val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, 
spark, sql, streaming,
+  streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, 
streamingZeromq) =
+Seq(bagel, catalyst, core, graphx, hive, mllib, repl, 
spark, sql,
+  streaming, streaming-flume, streaming-kafka, streaming-mqtt, 
streaming-twitter,
+  streaming-zeromq).map(ProjectRef(buildLocation, _))
 
-  // Hadoop version to build against. For example, 1.0.4 for Apache 
releases, or
-  // 2.0.0-mr1-cdh4.2.0 for Cloudera Hadoop. Note that these variables 
can be set
-  // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN.
-  val DEFAULT_HADOOP_VERSION = 1.0.4
+  val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, 
java8Tests, sparkGangliaLgpl) =
+Seq(yarn, yarn-stable, yarn-alpha, java8-tests, 
ganglia-lgpl).map(ProjectRef(buildLocation, _))
--- End diff --

This line is  100 characters.


---
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: [SPARK-1516]Throw exception in yarn client ins...

2014-07-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1099#issuecomment-47982776
  
Merged build started. 


---
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.
---


  1   2   3   >