[GitHub] spark pull request: [SPARK-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#discussion_r11012011
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/input/WholeTextFileInputFormat.scala
 ---
@@ -0,0 +1,47 @@
+/*
+ * 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.input
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.InputSplit
+import org.apache.hadoop.mapreduce.JobContext
+import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat
+import org.apache.hadoop.mapreduce.RecordReader
+import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit
+
+/**
+ * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat 
CombineFileInputFormat]] for
+ * reading whole text files. Each file is read as key-value pair, where 
the key is the file path and
+ * the value is the entire content of file.
+ */
+
+private[mllib] class WholeTextFileInputFormat extends 
CombineFileInputFormat[String, String] {
+  override protected def isSplitable(context: JobContext, file: Path): 
Boolean = false
+
+  override def createRecordReader(
+  split: InputSplit,
+  context: TaskAttemptContext): RecordReader[String, String] = {
+
+new CombineFileRecordReader[String, String](
+split.asInstanceOf[CombineFileSplit],
--- End diff --

extra 2 spaces


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012023
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala 
---
@@ -44,6 +44,16 @@ trait Row extends Seq[Any] with Serializable {
 s"[${this.mkString(",")}]"
 
   def copy(): Row
+
+  /** Returns true if there are any NULL values in this row. */
+  def anyNull: Boolean = {
+var i = 0
+while(i < length) {
--- End diff --

space after while and if...


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012029
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala 
---
@@ -44,6 +44,16 @@ trait Row extends Seq[Any] with Serializable {
 s"[${this.mkString(",")}]"
 
   def copy(): Row
+
+  /** Returns true if there are any NULL values in this row. */
+  def anyNull: Boolean = {
+var i = 0
+while(i < length) {
+  if(isNullAt(i)) return true
--- End diff --

and add { } even for one liner ...


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012038
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -15,23 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.execution
+package org.apache.spark.sql
+package execution
 
-import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, BitSet}
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext
 
-import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
-import 
org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, 
Partitioning}
+import catalyst.errors._
--- End diff --

let's not undo cheng's changes :)


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012037
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -15,23 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.execution
+package org.apache.spark.sql
--- End diff --

let's not undo cheng's changes :)


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012078
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -15,23 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.execution
+package org.apache.spark.sql
+package execution
 
-import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, BitSet}
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext
 
-import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
-import 
org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, 
Partitioning}
+import catalyst.errors._
+import catalyst.expressions._
+import catalyst.plans._
+import catalyst.plans.physical.{ClusteredDistribution, Partitioning}
 
-import org.apache.spark.rdd.PartitionLocalRDDFunctions._
+sealed abstract class BuildSide
+case object BuildLeft extends BuildSide
+case object BuildRight extends BuildSide
 
-case class SparkEquiInnerJoin(
+object InterpretCondition {
+  def apply(expression: Expression): (Row => Boolean) = {
--- End diff --

Do you think we will use this multiple times? If not.. maybe just put it 
inline where it is used? The signature of this whole function is not super easy 
to read


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38774514
  
 Merged build triggered. One or more automated tests failed


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38774524
  
Merged build started. One or more automated tests failed


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012103
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -95,17 +158,19 @@ case class BroadcastNestedLoopJoin(
   def right = broadcast
 
   @transient lazy val boundCondition =
-condition
-  .map(c => BindReferences.bindReference(c, left.output ++ 
right.output))
-  .getOrElse(Literal(true))
+InterpretCondition(
+  condition
+.map(c => BindReferences.bindReference(c, left.output ++ 
right.output))
+.getOrElse(Literal(true)))
 
 
   def execute() = {
 val broadcastedRelation = 
sc.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq)
 
 val streamedPlusMatches = streamed.execute().mapPartitions { 
streamedIter =>
-  val matchedRows = new mutable.ArrayBuffer[Row]
-  val includedBroadcastTuples =  new 
mutable.BitSet(broadcastedRelation.value.size)
+  val matchedRows = new ArrayBuffer[Row]
+  val includedBroadcastTuples =
+new scala.collection.mutable.BitSet(broadcastedRelation.value.size)
--- End diff --

Use the bitset here: org.apache.spark.util.collection.BitSet

It is way faster than Scala'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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012135
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
--- End diff --

Use the hash map here: org.apache.spark.util.collection.OpenHashMap

It is way faster than Java's, uses less memory, and the changeValue 
function allows you to use a single lookup to do lookup and update the value 
(so you can avoid the two hash lookups)



---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012174
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -15,23 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.execution
+package org.apache.spark.sql
+package execution
 
-import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, BitSet}
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext
 
-import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
-import 
org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, 
Partitioning}
+import catalyst.errors._
+import catalyst.expressions._
+import catalyst.plans._
+import catalyst.plans.physical.{ClusteredDistribution, Partitioning}
 
-import org.apache.spark.rdd.PartitionLocalRDDFunctions._
+sealed abstract class BuildSide
+case object BuildLeft extends BuildSide
+case object BuildRight extends BuildSide
 
-case class SparkEquiInnerJoin(
+object InterpretCondition {
+  def apply(expression: Expression): (Row => Boolean) = {
+(r: Row) => expression.apply(r).asInstanceOf[Boolean]
+  }
+}
+
+case class HashJoin(
--- End diff --

Seems to me this only handles inner join right? If yes, maybe reflect that 
in the name. Technically we can do outer joins with hash join too ...


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012223
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
+currentRow = buildIter.next()
+val rowKey = buildSideKeyGenerator(currentRow)
+if(!rowKey.anyNull) {
+  val existingMatchList = hashTable.get(rowKey)
+  val matchList = if (existingMatchList == null) {
+val newMatchList = new ArrayBuffer[Row]()
+hashTable.put(rowKey, newMatchList)
+newMatchList
+  } else {
+existingMatchList
+  }
+  matchList += currentRow.copy()
+}
+  }
 
-  /**
-   * Filters any rows where the any of the join keys is null, ensuring 
three-valued
-   * logic for the equi-join conditions.
-   */
-  protected def filterNulls(rdd: RDD[(Row, Row)]) =
-rdd.filter {
-  case (key: Seq[_], _) => !key.exists(_ == null)
+  new Iterator[Row] {
+private[this] var currentRow: Row = _
+private[this] var currentMatches: ArrayBuffer[Row] = _
+private[this] var currentPosition: Int = -1
+
+// Mutable per row objects.
+private[this] val joinRow = new JoinedRow
+
+@transient private val joinKeys = streamSideKeyGenerator()
+
+def hasNext: Boolean =
+  (currentPosition != -1 && currentPosition < currentMatches.size) 
||
+  (streamIter.hasNext && fetchNext())
+
+def next() = {
+  val ret = joinRow(currentRow, currentMatches(currentPosition))
+  currentPosition += 1
+  ret
+}
+
+private def fetchNext(): Boolean = {
--- End diff --

Would be helpful if u document inline what the return value does to future 
readers (i.e. true if there is another tuple to be fetched)


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012293
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
+currentRow = buildIter.next()
+val rowKey = buildSideKeyGenerator(currentRow)
+if(!rowKey.anyNull) {
+  val existingMatchList = hashTable.get(rowKey)
+  val matchList = if (existingMatchList == null) {
+val newMatchList = new ArrayBuffer[Row]()
+hashTable.put(rowKey, newMatchList)
+newMatchList
+  } else {
+existingMatchList
+  }
+  matchList += currentRow.copy()
+}
+  }
 
-  /**
-   * Filters any rows where the any of the join keys is null, ensuring 
three-valued
-   * logic for the equi-join conditions.
-   */
-  protected def filterNulls(rdd: RDD[(Row, Row)]) =
-rdd.filter {
-  case (key: Seq[_], _) => !key.exists(_ == null)
+  new Iterator[Row] {
+private[this] var currentRow: Row = _
--- End diff --

add inline comment to explain these 3 variables. the naming is slightly 
confusing right now because they are all "current", while "currentRow" refers 
to the streaming side, and currentMatches refers to the hash side, and 
currentPosition refers to the position in currentMatches ... I think adding 
some inline comment would make them clear.


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012318
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
+currentRow = buildIter.next()
+val rowKey = buildSideKeyGenerator(currentRow)
+if(!rowKey.anyNull) {
+  val existingMatchList = hashTable.get(rowKey)
+  val matchList = if (existingMatchList == null) {
+val newMatchList = new ArrayBuffer[Row]()
+hashTable.put(rowKey, newMatchList)
+newMatchList
+  } else {
+existingMatchList
+  }
+  matchList += currentRow.copy()
+}
+  }
 
-  /**
-   * Filters any rows where the any of the join keys is null, ensuring 
three-valued
-   * logic for the equi-join conditions.
-   */
-  protected def filterNulls(rdd: RDD[(Row, Row)]) =
-rdd.filter {
-  case (key: Seq[_], _) => !key.exists(_ == null)
+  new Iterator[Row] {
+private[this] var currentRow: Row = _
+private[this] var currentMatches: ArrayBuffer[Row] = _
+private[this] var currentPosition: Int = -1
+
+// Mutable per row objects.
+private[this] val joinRow = new JoinedRow
+
+@transient private val joinKeys = streamSideKeyGenerator()
+
+def hasNext: Boolean =
--- End diff --

add override to hasNext and next


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012314
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
+currentRow = buildIter.next()
+val rowKey = buildSideKeyGenerator(currentRow)
+if(!rowKey.anyNull) {
+  val existingMatchList = hashTable.get(rowKey)
+  val matchList = if (existingMatchList == null) {
+val newMatchList = new ArrayBuffer[Row]()
+hashTable.put(rowKey, newMatchList)
+newMatchList
+  } else {
+existingMatchList
+  }
+  matchList += currentRow.copy()
+}
+  }
 
-  /**
-   * Filters any rows where the any of the join keys is null, ensuring 
three-valued
-   * logic for the equi-join conditions.
-   */
-  protected def filterNulls(rdd: RDD[(Row, Row)]) =
-rdd.filter {
-  case (key: Seq[_], _) => !key.exists(_ == null)
+  new Iterator[Row] {
+private[this] var currentRow: Row = _
+private[this] var currentMatches: ArrayBuffer[Row] = _
+private[this] var currentPosition: Int = -1
+
+// Mutable per row objects.
+private[this] val joinRow = new JoinedRow
+
+@transient private val joinKeys = streamSideKeyGenerator()
--- End diff --

why transient and not private[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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012351
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
+currentRow = buildIter.next()
+val rowKey = buildSideKeyGenerator(currentRow)
+if(!rowKey.anyNull) {
+  val existingMatchList = hashTable.get(rowKey)
+  val matchList = if (existingMatchList == null) {
+val newMatchList = new ArrayBuffer[Row]()
+hashTable.put(rowKey, newMatchList)
+newMatchList
+  } else {
+existingMatchList
+  }
+  matchList += currentRow.copy()
+}
+  }
 
-  /**
-   * Filters any rows where the any of the join keys is null, ensuring 
three-valued
-   * logic for the equi-join conditions.
-   */
-  protected def filterNulls(rdd: RDD[(Row, Row)]) =
-rdd.filter {
-  case (key: Seq[_], _) => !key.exists(_ == null)
+  new Iterator[Row] {
+private[this] var currentRow: Row = _
+private[this] var currentMatches: ArrayBuffer[Row] = _
+private[this] var currentPosition: Int = -1
+
+// Mutable per row objects.
+private[this] val joinRow = new JoinedRow
+
+@transient private val joinKeys = streamSideKeyGenerator()
+
+def hasNext: Boolean =
--- End diff --

actually for readability, it would be nicer to expand hasNext and write it 
out with if/else's. It would make it more obvious what happens, e.g.
```scala
if (currentPosition != -1) {
  currentPosition < currentMatches.size
} else {
  fetchNext()   < looks to me u don't need to check 
streamIter.hasNext
}
```


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012360
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
+currentRow = buildIter.next()
+val rowKey = buildSideKeyGenerator(currentRow)
+if(!rowKey.anyNull) {
+  val existingMatchList = hashTable.get(rowKey)
+  val matchList = if (existingMatchList == null) {
+val newMatchList = new ArrayBuffer[Row]()
+hashTable.put(rowKey, newMatchList)
+newMatchList
+  } else {
+existingMatchList
+  }
+  matchList += currentRow.copy()
+}
+  }
 
-  /**
-   * Filters any rows where the any of the join keys is null, ensuring 
three-valued
-   * logic for the equi-join conditions.
-   */
-  protected def filterNulls(rdd: RDD[(Row, Row)]) =
-rdd.filter {
-  case (key: Seq[_], _) => !key.exists(_ == null)
+  new Iterator[Row] {
+private[this] var currentRow: Row = _
+private[this] var currentMatches: ArrayBuffer[Row] = _
+private[this] var currentPosition: Int = -1
+
+// Mutable per row objects.
+private[this] val joinRow = new JoinedRow
+
+@transient private val joinKeys = streamSideKeyGenerator()
+
+def hasNext: Boolean =
+  (currentPosition != -1 && currentPosition < currentMatches.size) 
||
+  (streamIter.hasNext && fetchNext())
+
+def next() = {
+  val ret = joinRow(currentRow, currentMatches(currentPosition))
+  currentPosition += 1
+  ret
+}
+
+private def fetchNext(): Boolean = {
+  currentMatches = null
+  currentPosition = -1
+
+  while (currentMatches == null && streamIter.hasNext) {
+currentRow = streamIter.next()
+if(!joinKeys(currentRow).anyNull) {
--- End diff --

space after if


---
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-1305: Support persisting RDD's directly ...

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

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


---
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: [SQL] Rewrite join implementation to allow str...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/250#discussion_r11012376
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala ---
@@ -40,33 +50,86 @@ case class SparkEquiInnerJoin(
   override def requiredChildDistribution =
 ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
 
+  val (buildPlan, streamedPlan) = buildSide match {
+case BuildLeft => (left, right)
+case BuildRight => (right, left)
+  }
+
+  val (buildKeys, streamedKeys) = buildSide match {
+case BuildLeft => (leftKeys, rightKeys)
+case BuildRight => (rightKeys, leftKeys)
+  }
+
   def output = left.output ++ right.output
 
-  def execute() = attachTree(this, "execute") {
-val leftWithKeys = left.execute().mapPartitions { iter =>
-  val generateLeftKeys = new Projection(leftKeys, left.output)
-  iter.map(row => (generateLeftKeys(row), row.copy()))
-}
+  @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, 
buildPlan.output)
+  @transient lazy val streamSideKeyGenerator =
+() => new MutableProjection(streamedKeys, streamedPlan.output)
 
-val rightWithKeys = right.execute().mapPartitions { iter =>
-  val generateRightKeys = new Projection(rightKeys, right.output)
-  iter.map(row => (generateRightKeys(row), row.copy()))
-}
+  def execute() = {
 
-// Do the join.
-val joined = 
filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
-// Drop join keys and merge input tuples.
-joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple 
++ rightTuple) }
-  }
+buildPlan.execute().zipPartitions(streamedPlan.execute()) { 
(buildIter, streamIter) =>
+  val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+  var currentRow: Row = null
+
+  // Create a mapping of buildKeys -> rows
+  while(buildIter.hasNext) {
--- End diff --

space after while / if


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38775432
  
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-1268] Adding XOR and AND-NOT operations...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/172#discussion_r11012411
  
--- Diff: core/src/main/scala/org/apache/spark/util/collection/BitSet.scala 
---
@@ -88,6 +88,46 @@ class BitSet(numBits: Int) extends Serializable {
 newBS
   }
 
+
+  /**
+   * Compute the symmetric difference by performing bit-wise XOR of the 
two sets returning the
+   * result.
+   */
+  def ^(other: BitSet): BitSet = {
+val newBS = new BitSet(math.max(capacity, other.capacity))
+val smaller = math.min(numWords, other.numWords)
+var ind = 0
+while ( ind < smaller ) {
--- End diff --

remove the space within parenthesis, i.e.

```scala
while (ind < smaller) {
```

this should apply to all the if / whiles in this diff


---
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-1268] Adding XOR and AND-NOT operations...

2014-03-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/172#discussion_r11012402
  
--- Diff: core/src/main/scala/org/apache/spark/util/collection/BitSet.scala 
---
@@ -88,6 +88,46 @@ class BitSet(numBits: Int) extends Serializable {
 newBS
   }
 
+
--- End diff --

do you mind removing the extra line here?


---
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-1268] Adding XOR and AND-NOT operations...

2014-03-27 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/172#issuecomment-38775516
  
Hey @petko-nikolov 

thanks for updating it. This looks pretty good to me. Just had a couple 
more comments on coding style. It would be great if you can update it, and then 
we can include it in the 1.0.0 release of Spark. 


---
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-1133] Add whole text files reader in ML...

2014-03-27 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/252#issuecomment-38775575
  
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-1305: Support persisting RDD's directly ...

2014-03-27 Thread haoyuan
Github user haoyuan commented on the pull request:

https://github.com/apache/spark/pull/158#issuecomment-38775628
  
Matei, 

1) Is there any way to have the unit test for this on by default? What's 
required to set it up? This is a pretty major piece of functionality and it 
would be good to test it by default.

It requires to run Tachyon locally to set it up.

I totally agree to test it by default. Tachyon has a local cluster mode for 
testing. But it is in the test jar, which will be release from Tachyon 0.5.0. 
The current unit test has been tested locally. How about let's modify this to 
use TachyonLocalClusterMode after from 0.5.0 release?


---
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-1057: Upgrade fastutil to 6.5.11

2014-03-27 Thread srowen
Github user srowen commented on the pull request:

https://github.com/apache/spark/pull/215#issuecomment-38775656
  
FWIW +1, fastutil is a beast. I volunteer to take a crack at the necessary 
change to understand what it looks like, if anyone will second the motion.


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38775654
  
 Merged build triggered. One or more automated tests failed


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38775661
  
Merged build started. One or more automated tests failed


---
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-1133] Add whole text files reader in ML...

2014-03-27 Thread yinxusen
Github user yinxusen commented on the pull request:

https://github.com/apache/spark/pull/252#issuecomment-38775740
  
Seems that it is running & testing in background: 
https://travis-ci.org/apache/spark/builds/21653655


---
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-1325. The maven build error for Spark To...

2014-03-27 Thread srowen
Github user srowen commented on the pull request:

https://github.com/apache/spark/pull/240#issuecomment-38775770
  
Yes @witgo deserves the credit of course.
I agree there is a version inconsistency here although it's a separate 
issue. I was actually preparing a different PR to address a few things like 
that and can include this change in there as 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: SPARK-1057: Upgrade fastutil to 6.5.11

2014-03-27 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/215#issuecomment-38775806
  
I third the motion!


---
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-1057: Upgrade fastutil to 6.5.11

2014-03-27 Thread velvia
Github user velvia commented on the pull request:

https://github.com/apache/spark/pull/215#issuecomment-38775813
  
So I was just looking at this.   fastutil is used in about 15 different
files, and mostly what is used is:
- A few of the hashmaps
- import it.unimi.dsi.fastutil.io.FastBufferedInputStream
- FastByteArrayOutputStream

I totally agree fastutil is a huge dep, but we only have a week left for
merge window -- is there time to find a comparable replacement for the
Fast*Stream stuff?

-Evan



On Wed, Mar 26, 2014 at 6:58 PM, Matei Zaharia 
wrote:

> I wonder if we should just get rid of FastUtil -- it's a big dependency
> (due to having lots of specialized classes) and we only use it in a few
> places.
>
> --
> Reply to this email directly or view it on 
GitHub
> .
>



-- 
The fruit of silence is prayer;
the fruit of prayer is faith;
the fruit of faith is love;
the fruit of love is service;
the fruit of service is peace.  -- Mother Teresa


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38775901
  
 Build triggered. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38775909
  
Build started. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

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


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38775964
  
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-1325. The maven build error for Spark To...

2014-03-27 Thread witgo
Github user witgo commented on the pull request:

https://github.com/apache/spark/pull/240#issuecomment-38776405
  
Uh, create a different PR is a good idea


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38776956
  
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: [SQL] Rewrite join implementation to allow str...

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

https://github.com/apache/spark/pull/250#issuecomment-38776957
  
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: [SQL] Rewrite join implementation to allow str...

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

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


---
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-1133] Add whole text files reader in ML...

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

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


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38778463
  
Build started. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38778453
  
 Build triggered. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38778455
  
 Merged build triggered. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38778465
  
Merged build started. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

2014-03-27 Thread yinxusen
Github user yinxusen commented on a diff in the pull request:

https://github.com/apache/spark/pull/245#discussion_r11013629
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala ---
@@ -26,17 +28,26 @@ abstract class Gradient extends Serializable {
   /**
* Compute the gradient and loss given the features of a single data 
point.
*
-   * @param data - Feature values for one data point. Column matrix of 
size dx1
-   *   where d is the number of features.
-   * @param label - Label for this data item.
-   * @param weights - Column matrix containing weights for every feature.
+   * @param data features for one data point
+   * @param label label for this data point
+   * @param weights weights/coefficients corresponding to features
*
-   * @return A tuple of 2 elements. The first element is a column matrix 
containing the computed
-   * gradient and the second element is the loss computed at this 
data point.
+   * @return (gradient: Vector, loss: Double)
+   */
+  def compute(data: Vector, label: Double, weights: Vector): (Vector, 
Double)
+
+  /**
+   * Compute the gradient and loss given the features of a single data 
point, add the gradient to a provided vector to
--- End diff --

100 characters limit.


---
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-1212, Part II] [WIP] Support sparse dat...

2014-03-27 Thread yinxusen
Github user yinxusen commented on a diff in the pull request:

https://github.com/apache/spark/pull/245#discussion_r11013706
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala ---
@@ -57,7 +68,22 @@ class LogisticGradient extends Gradient {
 math.log(1 + math.exp(margin)) - margin
   }
 
-(gradient, loss)
+(Vectors.fromBreeze(gradient), loss)
+  }
+
+  override def compute(data: Vector, label: Double, weights: Vector, 
gradientAddTo: Vector): Double = {
+val brzData = data.toBreeze
+val brzWeights = weights.toBreeze
+val margin: Double = -1.0 * brzWeights.dot(brzData)
+val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label
+
+brzAxpy(gradientMultiplier, brzData, gradientAddTo.toBreeze)
--- End diff --

I think there are too many `toBreeze`s when using the `Vector` trait. How 
about using `implicit` to eliminate them?


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38778747
  
Build finished. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38778749
  
Merged build started. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38778748
  
One or more automated tests failed
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13508/


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38778740
  
 Merged build triggered. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38779158
  
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-1133] Add whole text files reader in ML...

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

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


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38779159
  
One or more automated tests failed
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13505/


---
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-1305: Support persisting RDD's directly ...

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

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


---
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38779157
  
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-1133] Add whole text files reader in ML...

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

https://github.com/apache/spark/pull/252#issuecomment-38779156
  
Merged build finished. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

2014-03-27 Thread RongGu
Github user RongGu commented on the pull request:

https://github.com/apache/spark/pull/158#issuecomment-38779460
  
Hey @mateiz ,
2) What happens if someone creates a StorageLevel with useTachyon = true 
and replication > 1? Do we get two files? Do they step over each other?
3) Related to 2, in the first version we might want to let users put 
useTachyon only if useMemory and useDisk are false (and possibly the same with 
replication). That would reduce the number of weird interactions. I don't see 
when you'd want both Tachyon and disk for example, and furthermore Tachyon 
doesn't notify us when something falls out of it.

Currently, when a user creates a StorageLevel with useTachyon = true and 
replication = n, the rdd will be stored with n replicas explicitly. They will 
be stored in n executors' directories on Tachyon respectively. They won't step 
over each other.



---
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-1212, Part II] [WIP] Support sparse dat...

2014-03-27 Thread yinxusen
Github user yinxusen commented on a diff in the pull request:

https://github.com/apache/spark/pull/245#discussion_r11013983
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala ---
@@ -67,44 +70,50 @@ class RidgeRegressionWithSGD private (
 .setMiniBatchFraction(miniBatchFraction)
 
   // We don't want to penalize the intercept in RidgeRegression, so set 
this to false.
-  setIntercept(false)
+  super.setIntercept(false)
 
   var yMean = 0.0
-  var xColMean: DoubleMatrix = _
-  var xColSd: DoubleMatrix = _
+  var xColMean: BV[Double] = _
+  var xColSd: BV[Double] = _
--- End diff --

add `private` modifier like in `LassoRegression`?


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38780054
  
 Build triggered. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38780064
  
Build started. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38780065
  
Build started. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38780052
  
 Build triggered. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38780181
  
Build finished. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38780182
  
One or more automated tests failed
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13511/


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#discussion_r11014289
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala ---
@@ -67,44 +70,50 @@ class RidgeRegressionWithSGD private (
 .setMiniBatchFraction(miniBatchFraction)
 
   // We don't want to penalize the intercept in RidgeRegression, so set 
this to false.
-  setIntercept(false)
+  super.setIntercept(false)
 
   var yMean = 0.0
-  var xColMean: DoubleMatrix = _
-  var xColSd: DoubleMatrix = _
+  var xColMean: BV[Double] = _
+  var xColSd: BV[Double] = _
--- End diff --

yes


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#discussion_r11014277
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala ---
@@ -57,7 +68,22 @@ class LogisticGradient extends Gradient {
 math.log(1 + math.exp(margin)) - margin
   }
 
-(gradient, loss)
+(Vectors.fromBreeze(gradient), loss)
+  }
+
+  override def compute(data: Vector, label: Double, weights: Vector, 
gradientAddTo: Vector): Double = {
+val brzData = data.toBreeze
+val brzWeights = weights.toBreeze
+val margin: Double = -1.0 * brzWeights.dot(brzData)
+val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label
+
+brzAxpy(gradientMultiplier, brzData, gradientAddTo.toBreeze)
--- End diff --

breeze uses implicits a lot. Scala do not look for second degree implicit 
conversions.


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38780484
  
@yinxusen This is WIP. I will let you know when it is ready for review.


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38781785
  
Build started. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38781772
  
 Build triggered. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38782189
  
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38782164
  
Merged build started. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38782146
  
 Merged build triggered. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

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


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38782530
  
Build finished. One or more automated tests failed


---
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-1212, Part II] [WIP] Support sparse dat...

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

https://github.com/apache/spark/pull/245#issuecomment-38782531
  
One or more automated tests failed
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13513/


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38782622
  
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-1096, a space after comment start style ...

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

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


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38783643
  
Build started. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38783630
  
 Build triggered. One or more automated tests failed


---
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-1335. Also increase perm gen / code cach...

2014-03-27 Thread srowen
GitHub user srowen opened a pull request:

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

SPARK-1335. Also increase perm gen / code cache for scalatest when invoked 
via Maven build

I am observing build failures when the Maven build reaches tests in the new 
SQL components. (I'm on Java 7 / OSX 10.9). The failure is the usual complaint 
from scala, that it's out of permgen space, or that JIT out of code cache space.

I see that various build scripts increase these both for SBT. This change 
simply adds these settings to scalatest's arguments. Works for me and seems a 
bit more consistent.

(I also snuck in cures for new build warnings from new scaladoc. Felt too 
trivial for a new PR, although it's separate. Just something I also saw while 
examining the build output.)

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

$ git pull https://github.com/srowen/spark SPARK-1335

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

https://github.com/apache/spark/pull/253.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 #253


commit b2c6a099b9b1dd990f024a7657ce0ba53761b2c7
Author: Sean Owen 
Date:   2014-03-27T10:10:55Z

Add perm gen, code cache settings to scalatest, mirroring SBT settings 
elsewhere, which allows tests to complete in at least one environment where 
they are failing. (Also removed a duplicate -Xms setting elsewhere.)

commit a02679cf15f36d652f48ceb4deb1a18e0bb10f23
Author: Sean Owen 
Date:   2014-03-27T10:11:41Z

Fix scaladoc errors due to missing links, which are generating build 
warnings, from some recent doc changes. We apparently can't generate links 
outside the module.




---
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38786607
  
 Merged build triggered. One or more automated tests failed


---
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38786617
  
Merged build started. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38786879
  
One or more automated tests failed
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13514/


---
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38786832
  
One or more automated tests failed
Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13516/


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38786877
  
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38786830
  
Merged build finished. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

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


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38786878
  
Merged build finished. One or more automated tests failed


---
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38787822
  
Merged build started. One or more automated tests failed


---
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38787812
  
 Merged build triggered. One or more automated tests failed


---
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-1305: Support persisting RDD's directly ...

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

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


---
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-1305: Support persisting RDD's directly ...

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

https://github.com/apache/spark/pull/158#issuecomment-38788130
  
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-1268] Adding XOR and AND-NOT operations...

2014-03-27 Thread petko-nikolov
Github user petko-nikolov commented on a diff in the pull request:

https://github.com/apache/spark/pull/172#discussion_r11018500
  
--- Diff: core/src/main/scala/org/apache/spark/util/collection/BitSet.scala 
---
@@ -88,6 +88,46 @@ class BitSet(numBits: Int) extends Serializable {
 newBS
   }
 
+
+  /**
+   * Compute the symmetric difference by performing bit-wise XOR of the 
two sets returning the
+   * result.
+   */
+  def ^(other: BitSet): BitSet = {
+val newBS = new BitSet(math.max(capacity, other.capacity))
+val smaller = math.min(numWords, other.numWords)
+var ind = 0
+while ( ind < smaller ) {
--- End diff --

Thanks! I should be more careful with the coding style. Is there a way to 
check for such stylistic mistakes? I ran scalastyle but it didn't show any 
errors.


---
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-1268] Adding XOR and AND-NOT operations...

2014-03-27 Thread petko-nikolov
Github user petko-nikolov commented on a diff in the pull request:

https://github.com/apache/spark/pull/172#discussion_r11018505
  
--- Diff: core/src/main/scala/org/apache/spark/util/collection/BitSet.scala 
---
@@ -88,6 +88,46 @@ class BitSet(numBits: Int) extends Serializable {
 newBS
   }
 
+
--- End diff --

done :)


---
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-1335. Also increase perm gen / code cach...

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

https://github.com/apache/spark/pull/253#issuecomment-38791761
  
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-1335. Also increase perm gen / code cach...

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

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


---
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-1096, a space after comment start style ...

2014-03-27 Thread ScrapCodes
Github user ScrapCodes commented on the pull request:

https://github.com/apache/spark/pull/124#issuecomment-38793369
  
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38793417
  
 Merged build triggered. One or more automated tests failed


---
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-1096, a space after comment start style ...

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

https://github.com/apache/spark/pull/124#issuecomment-38793427
  
Merged build started. One or more automated tests failed


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